Merge branch 'master' into ADQM-626

This commit is contained in:
Roman Vasin 2022-12-04 20:34:55 +03:00 committed by GitHub
commit a09e7fddcc
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
233 changed files with 4856 additions and 2166 deletions

1
.exrc Normal file
View File

@ -0,0 +1 @@
au BufRead,BufNewFile * set tabstop=4 softtabstop=0 expandtab shiftwidth=4 smarttab tags=tags,../tags

2
.vimrc
View File

@ -1,2 +0,0 @@
au BufRead,BufNewFile ./* set tabstop=4 softtabstop=0 expandtab shiftwidth=4 smarttab tags=tags,../tags

2
contrib/NuRaft vendored

@ -1 +1 @@
Subproject commit e4e746a24eb56861a86f3672771e3308d8c40722 Subproject commit afc36dfa9b0beb45bc4cd935060631cc80ba04a5

View File

@ -295,6 +295,9 @@ if not args.use_existing_tables:
reportStageEnd("create") reportStageEnd("create")
# Let's sync the data to avoid writeback affects performance
os.system("sync")
# By default, test all queries. # By default, test all queries.
queries_to_run = range(0, len(test_queries)) queries_to_run = range(0, len(test_queries))

View File

@ -17,7 +17,7 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \
python3-pip \ python3-pip \
shellcheck \ shellcheck \
yamllint \ yamllint \
&& pip3 install black==22.8.0 boto3 codespell==2.2.1 dohq-artifactory PyGithub unidiff pylint==2.6.2 \ && pip3 install black==22.8.0 boto3 codespell==2.2.1 dohq-artifactory mypy PyGithub unidiff pylint==2.6.2 \
&& apt-get clean \ && apt-get clean \
&& rm -rf /root/.cache/pip && rm -rf /root/.cache/pip

View File

@ -11,17 +11,19 @@ def process_result(result_folder):
description = "" description = ""
test_results = [] test_results = []
checks = ( checks = (
("header duplicates", "duplicate_output.txt"), "duplicate includes",
("shellcheck", "shellcheck_output.txt"), "shellcheck",
("style", "style_output.txt"), "style",
("black", "black_output.txt"), "black",
("typos", "typos_output.txt"), "mypy",
("whitespaces", "whitespaces_output.txt"), "typos",
("workflows", "workflows_output.txt"), "whitespaces",
("doc typos", "doc_spell_output.txt"), "workflows",
"docs spelling",
) )
for name, out_file in checks: for name in checks:
out_file = name.replace(" ", "_") + "_output.txt"
full_path = os.path.join(result_folder, out_file) full_path = os.path.join(result_folder, out_file)
if not os.path.exists(full_path): if not os.path.exists(full_path):
logging.info("No %s check log on path %s", name, full_path) logging.info("No %s check log on path %s", name, full_path)

View File

@ -4,15 +4,17 @@
cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_output/check_status.tsv cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_output/check_status.tsv
echo "Check duplicates" | ts echo "Check duplicates" | ts
./check-duplicate-includes.sh |& tee /test_output/duplicate_output.txt ./check-duplicate-includes.sh |& tee /test_output/duplicate_includes_output.txt
echo "Check style" | ts echo "Check style" | ts
./check-style -n |& tee /test_output/style_output.txt ./check-style -n |& tee /test_output/style_output.txt
echo "Check python formatting with black" | ts echo "Check python formatting with black" | ts
./check-black -n |& tee /test_output/black_output.txt ./check-black -n |& tee /test_output/black_output.txt
echo "Check python type hinting with mypy" | ts
./check-mypy -n |& tee /test_output/mypy_output.txt
echo "Check typos" | ts echo "Check typos" | ts
./check-typos |& tee /test_output/typos_output.txt ./check-typos |& tee /test_output/typos_output.txt
echo "Check docs spelling" | ts echo "Check docs spelling" | ts
./check-doc-aspell |& tee /test_output/doc_spell_output.txt ./check-doc-aspell |& tee /test_output/docs_spelling_output.txt
echo "Check whitespaces" | ts echo "Check whitespaces" | ts
./check-whitespaces -n |& tee /test_output/whitespaces_output.txt ./check-whitespaces -n |& tee /test_output/whitespaces_output.txt
echo "Check workflows" | ts echo "Check workflows" | ts

View File

@ -1456,6 +1456,10 @@ If setting [input_format_with_types_use_header](../operations/settings/settings.
the types from input data will be compared with the types of the corresponding columns from the table. Otherwise, the second row will be skipped. the types from input data will be compared with the types of the corresponding columns from the table. Otherwise, the second row will be skipped.
::: :::
## RowBinary format settings {#row-binary-format-settings}
- [format_binary_max_string_size](../operations/settings/settings.md#format_binary_max_string_size) - The maximum allowed size for String in RowBinary format. Default value - `1GiB`.
## Values {#data-format-values} ## Values {#data-format-values}
Prints every row in brackets. Rows are separated by commas. There is no comma after the last row. The values inside the brackets are also comma-separated. Numbers are output in a decimal format without quotes. Arrays are output in square brackets. Strings, dates, and dates with times are output in quotes. Escaping rules and parsing are similar to the [TabSeparated](#tabseparated) format. During formatting, extra spaces arent inserted, but during parsing, they are allowed and skipped (except for spaces inside array values, which are not allowed). [NULL](../sql-reference/syntax.md) is represented as `NULL`. Prints every row in brackets. Rows are separated by commas. There is no comma after the last row. The values inside the brackets are also comma-separated. Numbers are output in a decimal format without quotes. Arrays are output in square brackets. Strings, dates, and dates with times are output in quotes. Escaping rules and parsing are similar to the [TabSeparated](#tabseparated) format. During formatting, extra spaces arent inserted, but during parsing, they are allowed and skipped (except for spaces inside array values, which are not allowed). [NULL](../sql-reference/syntax.md) is represented as `NULL`.

View File

@ -11,6 +11,7 @@ Main cache types:
- `mark_cache` — Cache of marks used by table engines of the [MergeTree](../engines/table-engines/mergetree-family/mergetree.md) family. - `mark_cache` — Cache of marks used by table engines of the [MergeTree](../engines/table-engines/mergetree-family/mergetree.md) family.
- `uncompressed_cache` — Cache of uncompressed data used by table engines of the [MergeTree](../engines/table-engines/mergetree-family/mergetree.md) family. - `uncompressed_cache` — Cache of uncompressed data used by table engines of the [MergeTree](../engines/table-engines/mergetree-family/mergetree.md) family.
- Operating system page cache (used indirectly, for files with actual data).
Additional cache types: Additional cache types:
@ -22,10 +23,4 @@ Additional cache types:
- Schema inference cache. - Schema inference cache.
- [Filesystem cache](storing-data.md) over S3, Azure, Local and other disks. - [Filesystem cache](storing-data.md) over S3, Azure, Local and other disks.
Indirectly used: To drop one of the caches, use [SYSTEM DROP ... CACHE](../sql-reference/statements/system.md#drop-mark-cache) statements.
- OS page cache.
To drop cache, use [SYSTEM DROP ... CACHE](../sql-reference/statements/system.md) statements.
[Original article](https://clickhouse.com/docs/en/operations/caches/) <!--hide-->

View File

@ -1807,6 +1807,41 @@ See also:
- System table [trace_log](../../operations/system-tables/trace_log.md/#system_tables-trace_log) - System table [trace_log](../../operations/system-tables/trace_log.md/#system_tables-trace_log)
## memory_profiler_step {#memory_profiler_step}
Sets the step of memory profiler. Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stacktrace and will write it into [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log).
Possible values:
- A positive integer number of bytes.
- 0 for turning off the memory profiler.
Default value: 4,194,304 bytes (4 MiB).
## memory_profiler_sample_probability {#memory_profiler_sample_probability}
Sets the probability of collecting stacktraces at random allocations and deallocations and writing them into [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log).
Possible values:
- A positive floating-point number in the range [0..1].
- 0.0 for turning off the memory sampling.
Default value: 0.0.
## trace_profile_events {#trace_profile_events}
Enables or disables collecting stacktraces on each update of profile events along with the name of profile event and the value of increment and sending them into [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log).
Possible values:
- 1 — Tracing of profile events enabled.
- 0 — Tracing of profile events disabled.
Default value: 0.
## allow_introspection_functions {#settings-allow_introspection_functions} ## allow_introspection_functions {#settings-allow_introspection_functions}
Enables or disables [introspections functions](../../sql-reference/functions/introspection.md) for query profiling. Enables or disables [introspections functions](../../sql-reference/functions/introspection.md) for query profiling.
@ -4829,3 +4864,11 @@ Disabled by default.
Allow skipping columns with unsupported types while schema inference for format BSONEachRow. Allow skipping columns with unsupported types while schema inference for format BSONEachRow.
Disabled by default. Disabled by default.
## RowBinary format settings {#row-binary-format-settings}
### format_binary_max_string_size {#format_binary_max_string_size}
The maximum allowed size for String in RowBinary format. It prevents allocating large amount of memory in case of corrupted data. 0 means there is no limit.
Default value: `1GiB`

View File

@ -5,7 +5,8 @@ slug: /en/operations/system-tables/trace_log
Contains stack traces collected by the sampling query profiler. Contains stack traces collected by the sampling query profiler.
ClickHouse creates this table when the [trace_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) server configuration section is set. Also the [query_profiler_real_time_period_ns](../../operations/settings/settings.md#query_profiler_real_time_period_ns) and [query_profiler_cpu_time_period_ns](../../operations/settings/settings.md#query_profiler_cpu_time_period_ns) settings should be set. ClickHouse creates this table when the [trace_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) server configuration section is set. Also see settings: [query_profiler_real_time_period_ns](../../operations/settings/settings.md#query_profiler_real_time_period_ns), [query_profiler_cpu_time_period_ns](../../operations/settings/settings.md#query_profiler_cpu_time_period_ns), [memory_profiler_step](../../operations/settings/settings.md#memory_profiler_step),
[memory_profiler_sample_probability](../../operations/settings/settings.md#memory_profiler_sample_probability), [trace_profile_events](../../operations/settings/settings.md#trace_profile_events).
To analyze logs, use the `addressToLine`, `addressToLineWithInlines`, `addressToSymbol` and `demangle` introspection functions. To analyze logs, use the `addressToLine`, `addressToLineWithInlines`, `addressToSymbol` and `demangle` introspection functions.
@ -29,6 +30,8 @@ Columns:
- `CPU` represents collecting stack traces by CPU time. - `CPU` represents collecting stack traces by CPU time.
- `Memory` represents collecting allocations and deallocations when memory allocation exceeds the subsequent watermark. - `Memory` represents collecting allocations and deallocations when memory allocation exceeds the subsequent watermark.
- `MemorySample` represents collecting random allocations and deallocations. - `MemorySample` represents collecting random allocations and deallocations.
- `MemoryPeak` represents collecting updates of peak memory usage.
- `ProfileEvent` represents collecting of increments of profile events.
- `thread_number` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Thread identifier. - `thread_number` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Thread identifier.
@ -36,6 +39,12 @@ Columns:
- `trace` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — Stack trace at the moment of sampling. Each element is a virtual memory address inside ClickHouse server process. - `trace` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — Stack trace at the moment of sampling. Each element is a virtual memory address inside ClickHouse server process.
- `size` ([Int64](../../sql-reference/data-types/int-uint.md)) - For trace types `Memory`, `MemorySample` or `MemoryPeak` is the amount of memory allocated, for other trace types is 0.
- `event` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) - For trace type `ProfileEvent` is the name of updated profile event, for other trace types is an empty string.
- `increment` ([UInt64](../../sql-reference/data-types/int-uint.md)) - For trace type `ProfileEvent` is the amount of incremnt of profile event, for other trace types is 0.
**Example** **Example**
``` sql ``` sql

View File

@ -32,8 +32,8 @@ The null hypothesis is that means of populations are equal. Normal distribution
- calculated t-statistic. [Float64](../../../sql-reference/data-types/float.md). - calculated t-statistic. [Float64](../../../sql-reference/data-types/float.md).
- calculated p-value. [Float64](../../../sql-reference/data-types/float.md). - calculated p-value. [Float64](../../../sql-reference/data-types/float.md).
- [calculated confidence-interval-low.] [Float64](../../../sql-reference/data-types/float.md). - calculated confidence-interval-low. [Float64](../../../sql-reference/data-types/float.md).
- [calculated confidence-interval-high.] [Float64](../../../sql-reference/data-types/float.md). - calculated confidence-interval-high. [Float64](../../../sql-reference/data-types/float.md).
**Example** **Example**

View File

@ -1865,6 +1865,17 @@ Next, specify the path to `libcatboostmodel.<so|dylib>` in the clickhouse config
</clickhouse> </clickhouse>
``` ```
For security and isolation reasons, the model evaluation does not run in the server process but in the clickhouse-library-bridge process.
At the first execution of `catboostEvaluate()`, the server starts the library bridge process if it is not running already. Both processes
communicate using a HTTP interface. By default, port `9012` is used. A different port can be specified as follows - this is useful if port
`9012` is already assigned to a different service.
``` xml
<library_bridge>
<port>9019</port>
</library_bridge>
```
2. Train a catboost model using libcatboost 2. Train a catboost model using libcatboost
See [Training and applying models](https://catboost.ai/docs/features/training.html#training) for how to train catboost models from a training data set. See [Training and applying models](https://catboost.ai/docs/features/training.html#training) for how to train catboost models from a training data set.

View File

@ -19,7 +19,7 @@ then
# Will make a repository with website content as the only commit. # Will make a repository with website content as the only commit.
git init git init
git remote add origin "${GIT_PROD_URI}" git remote add origin "${GIT_PROD_URI}"
git config user.email "robot-clickhouse@clickhouse.com" git config user.email "robot-clickhouse@users.noreply.github.com"
git config user.name "robot-clickhouse" git config user.name "robot-clickhouse"
# Add files. # Add files.

View File

@ -13,7 +13,6 @@ clickhouse_embed_binaries(
set(CLICKHOUSE_KEEPER_SOURCES set(CLICKHOUSE_KEEPER_SOURCES
Keeper.cpp Keeper.cpp
TinyContext.cpp
) )
set (CLICKHOUSE_KEEPER_LINK set (CLICKHOUSE_KEEPER_LINK
@ -49,6 +48,8 @@ if (BUILD_STANDALONE_KEEPER)
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateMachine.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateMachine.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateManager.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateManager.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStorage.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStorage.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperAsynchronousMetrics.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/TinyContext.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/pathUtils.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/pathUtils.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/SessionExpiryQueue.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/SessionExpiryQueue.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/SummingStateMachine.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/SummingStateMachine.cpp
@ -64,7 +65,18 @@ if (BUILD_STANDALONE_KEEPER)
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/KeeperTCPHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/KeeperTCPHandler.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/TCPServer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/TCPServer.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/NotFoundHandler.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/ProtocolServerAdapter.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/ProtocolServerAdapter.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/PrometheusRequestHandler.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/PrometheusMetricsWriter.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTPRequestHandlerFactoryMain.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServer.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/ReadHeaders.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServerConnection.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServerRequest.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServerResponse.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServerConnectionFactory.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CachedCompressedReadBuffer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CachedCompressedReadBuffer.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CheckingCompressedReadBuffer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/CheckingCompressedReadBuffer.cpp
@ -96,9 +108,7 @@ if (BUILD_STANDALONE_KEEPER)
${CMAKE_CURRENT_BINARY_DIR}/../../src/Daemon/GitHash.generated.cpp ${CMAKE_CURRENT_BINARY_DIR}/../../src/Daemon/GitHash.generated.cpp
Keeper.cpp Keeper.cpp
TinyContext.cpp
clickhouse-keeper.cpp clickhouse-keeper.cpp
) )
clickhouse_add_executable(clickhouse-keeper ${CLICKHOUSE_KEEPER_STANDALONE_SOURCES}) clickhouse_add_executable(clickhouse-keeper ${CLICKHOUSE_KEEPER_STANDALONE_SOURCES})

View File

@ -6,7 +6,6 @@
#include <Interpreters/DNSCacheUpdater.h> #include <Interpreters/DNSCacheUpdater.h>
#include <Coordination/Defines.h> #include <Coordination/Defines.h>
#include <Common/Config/ConfigReloader.h> #include <Common/Config/ConfigReloader.h>
#include <Server/TCPServer.h>
#include <filesystem> #include <filesystem>
#include <IO/UseSSL.h> #include <IO/UseSSL.h>
#include <Core/ServerUUID.h> #include <Core/ServerUUID.h>
@ -22,8 +21,15 @@
#include <Poco/Environment.h> #include <Poco/Environment.h>
#include <sys/stat.h> #include <sys/stat.h>
#include <pwd.h> #include <pwd.h>
#include <Coordination/FourLetterCommand.h>
#include <Coordination/FourLetterCommand.h>
#include <Coordination/KeeperAsynchronousMetrics.h>
#include <Server/HTTP/HTTPServer.h>
#include <Server/TCPServer.h>
#include <Server/HTTPHandlerFactory.h>
#include "Core/Defines.h"
#include "config.h" #include "config.h"
#include "config_version.h" #include "config_version.h"
@ -52,6 +58,16 @@ int mainEntryClickHouseKeeper(int argc, char ** argv)
} }
} }
#ifdef KEEPER_STANDALONE_BUILD
// Weak symbols don't work correctly on Darwin
// so we have a stub implementation to avoid linker errors
void collectCrashLog(
Int32, UInt64, const String &, const StackTrace &)
{}
#endif
namespace DB namespace DB
{ {
@ -261,6 +277,60 @@ void Keeper::defineOptions(Poco::Util::OptionSet & options)
BaseDaemon::defineOptions(options); BaseDaemon::defineOptions(options);
} }
struct Keeper::KeeperHTTPContext : public IHTTPContext
{
explicit KeeperHTTPContext(TinyContextPtr context_)
: context(std::move(context_))
{}
uint64_t getMaxHstsAge() const override
{
return context->getConfigRef().getUInt64("keeper_server.hsts_max_age", 0);
}
uint64_t getMaxUriSize() const override
{
return context->getConfigRef().getUInt64("keeper_server.http_max_uri_size", 1048576);
}
uint64_t getMaxFields() const override
{
return context->getConfigRef().getUInt64("keeper_server.http_max_fields", 1000000);
}
uint64_t getMaxFieldNameSize() const override
{
return context->getConfigRef().getUInt64("keeper_server.http_max_field_name_size", 1048576);
}
uint64_t getMaxFieldValueSize() const override
{
return context->getConfigRef().getUInt64("keeper_server.http_max_field_value_size", 1048576);
}
uint64_t getMaxChunkSize() const override
{
return context->getConfigRef().getUInt64("keeper_server.http_max_chunk_size", 100_GiB);
}
Poco::Timespan getReceiveTimeout() const override
{
return context->getConfigRef().getUInt64("keeper_server.http_receive_timeout", DEFAULT_HTTP_READ_BUFFER_TIMEOUT);
}
Poco::Timespan getSendTimeout() const override
{
return context->getConfigRef().getUInt64("keeper_server.http_send_timeout", DEFAULT_HTTP_READ_BUFFER_TIMEOUT);
}
TinyContextPtr context;
};
HTTPContextPtr Keeper::httpContext()
{
return std::make_shared<KeeperHTTPContext>(tiny_context);
}
int Keeper::main(const std::vector<std::string> & /*args*/) int Keeper::main(const std::vector<std::string> & /*args*/)
try try
{ {
@ -335,6 +405,25 @@ try
DNSResolver::instance().setDisableCacheFlag(); DNSResolver::instance().setDisableCacheFlag();
Poco::ThreadPool server_pool(3, config().getUInt("max_connections", 1024)); Poco::ThreadPool server_pool(3, config().getUInt("max_connections", 1024));
std::mutex servers_lock;
auto servers = std::make_shared<std::vector<ProtocolServerAdapter>>();
tiny_context = std::make_shared<TinyContext>();
/// This object will periodically calculate some metrics.
KeeperAsynchronousMetrics async_metrics(
tiny_context,
config().getUInt("asynchronous_metrics_update_period_s", 1),
[&]() -> std::vector<ProtocolServerMetrics>
{
std::vector<ProtocolServerMetrics> metrics;
std::lock_guard lock(servers_lock);
metrics.reserve(servers->size());
for (const auto & server : *servers)
metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()});
return metrics;
}
);
std::vector<std::string> listen_hosts = DB::getMultipleValuesFromConfig(config(), "", "listen_host"); std::vector<std::string> listen_hosts = DB::getMultipleValuesFromConfig(config(), "", "listen_host");
@ -346,15 +435,13 @@ try
listen_try = true; listen_try = true;
} }
auto servers = std::make_shared<std::vector<ProtocolServerAdapter>>();
/// Initialize keeper RAFT. Do nothing if no keeper_server in config. /// Initialize keeper RAFT. Do nothing if no keeper_server in config.
tiny_context.initializeKeeperDispatcher(/* start_async = */ true); tiny_context->initializeKeeperDispatcher(/* start_async = */ true);
FourLetterCommandFactory::registerCommands(*tiny_context.getKeeperDispatcher()); FourLetterCommandFactory::registerCommands(*tiny_context->getKeeperDispatcher());
auto config_getter = [this] () -> const Poco::Util::AbstractConfiguration & auto config_getter = [this] () -> const Poco::Util::AbstractConfiguration &
{ {
return tiny_context.getConfigRef(); return tiny_context->getConfigRef();
}; };
for (const auto & listen_host : listen_hosts) for (const auto & listen_host : listen_hosts)
@ -373,7 +460,7 @@ try
"Keeper (tcp): " + address.toString(), "Keeper (tcp): " + address.toString(),
std::make_unique<TCPServer>( std::make_unique<TCPServer>(
new KeeperTCPHandlerFactory( new KeeperTCPHandlerFactory(
config_getter, tiny_context.getKeeperDispatcher(), config_getter, tiny_context->getKeeperDispatcher(),
config().getUInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), config().getUInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC),
config().getUInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC), false), server_pool, socket)); config().getUInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC), false), server_pool, socket));
}); });
@ -392,7 +479,7 @@ try
"Keeper with secure protocol (tcp_secure): " + address.toString(), "Keeper with secure protocol (tcp_secure): " + address.toString(),
std::make_unique<TCPServer>( std::make_unique<TCPServer>(
new KeeperTCPHandlerFactory( new KeeperTCPHandlerFactory(
config_getter, tiny_context.getKeeperDispatcher(), config_getter, tiny_context->getKeeperDispatcher(),
config().getUInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), config().getUInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC),
config().getUInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC), true), server_pool, socket)); config().getUInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC), true), server_pool, socket));
#else #else
@ -401,6 +488,29 @@ try
ErrorCodes::SUPPORT_IS_DISABLED}; ErrorCodes::SUPPORT_IS_DISABLED};
#endif #endif
}); });
const auto & config = config_getter();
Poco::Timespan keep_alive_timeout(config.getUInt("keep_alive_timeout", 10), 0);
Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams;
http_params->setTimeout(DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC);
http_params->setKeepAliveTimeout(keep_alive_timeout);
/// Prometheus (if defined and not setup yet with http_port)
port_name = "prometheus.port";
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
{
Poco::Net::ServerSocket socket;
auto address = socketBindListen(socket, listen_host, port);
auto http_context = httpContext();
socket.setReceiveTimeout(http_context->getReceiveTimeout());
socket.setSendTimeout(http_context->getSendTimeout());
servers->emplace_back(
listen_host,
port_name,
"Prometheus: http://" + address.toString(),
std::make_unique<HTTPServer>(
std::move(http_context), createPrometheusMainHandlerFactory(*this, config_getter(), async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params));
});
} }
for (auto & server : *servers) for (auto & server : *servers)
@ -409,6 +519,8 @@ try
LOG_INFO(log, "Listening for {}", server.getDescription()); LOG_INFO(log, "Listening for {}", server.getDescription());
} }
async_metrics.start();
zkutil::EventPtr unused_event = std::make_shared<Poco::Event>(); zkutil::EventPtr unused_event = std::make_shared<Poco::Event>();
zkutil::ZooKeeperNodeCache unused_cache([] { return nullptr; }); zkutil::ZooKeeperNodeCache unused_cache([] { return nullptr; });
/// ConfigReloader have to strict parameters which are redundant in our case /// ConfigReloader have to strict parameters which are redundant in our case
@ -421,7 +533,7 @@ try
[&](ConfigurationPtr config, bool /* initial_loading */) [&](ConfigurationPtr config, bool /* initial_loading */)
{ {
if (config->has("keeper_server")) if (config->has("keeper_server"))
tiny_context.updateKeeperConfiguration(*config); tiny_context->updateKeeperConfiguration(*config);
}, },
/* already_loaded = */ false); /// Reload it right now (initial loading) /* already_loaded = */ false); /// Reload it right now (initial loading)
@ -429,6 +541,8 @@ try
LOG_INFO(log, "Shutting down."); LOG_INFO(log, "Shutting down.");
main_config_reloader.reset(); main_config_reloader.reset();
async_metrics.stop();
LOG_DEBUG(log, "Waiting for current connections to Keeper to finish."); LOG_DEBUG(log, "Waiting for current connections to Keeper to finish.");
size_t current_connections = 0; size_t current_connections = 0;
for (auto & server : *servers) for (auto & server : *servers)
@ -450,7 +564,7 @@ try
else else
LOG_INFO(log, "Closed connections to Keeper."); LOG_INFO(log, "Closed connections to Keeper.");
tiny_context.shutdownKeeperDispatcher(); tiny_context->shutdownKeeperDispatcher();
/// Wait server pool to avoid use-after-free of destroyed context in the handlers /// Wait server pool to avoid use-after-free of destroyed context in the handlers
server_pool.joinAll(); server_pool.joinAll();

View File

@ -1,8 +1,9 @@
#pragma once #pragma once
#include <Server/IServer.h> #include <Server/IServer.h>
#include <Server/HTTP/HTTPContext.h>
#include <Daemon/BaseDaemon.h> #include <Daemon/BaseDaemon.h>
#include "TinyContext.h" #include <Coordination/TinyContext.h>
namespace Poco namespace Poco
{ {
@ -15,29 +16,40 @@ namespace Poco
namespace DB namespace DB
{ {
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
/// standalone clickhouse-keeper server (replacement for ZooKeeper). Uses the same /// standalone clickhouse-keeper server (replacement for ZooKeeper). Uses the same
/// config as clickhouse-server. Serves requests on TCP ports with or without /// config as clickhouse-server. Serves requests on TCP ports with or without
/// SSL using ZooKeeper protocol. /// SSL using ZooKeeper protocol.
class Keeper : public BaseDaemon class Keeper : public BaseDaemon, public IServer
{ {
public: public:
using ServerApplication::run; using ServerApplication::run;
Poco::Util::LayeredConfiguration & config() const Poco::Util::LayeredConfiguration & config() const override
{ {
return BaseDaemon::config(); return BaseDaemon::config();
} }
Poco::Logger & logger() const Poco::Logger & logger() const override
{ {
return BaseDaemon::logger(); return BaseDaemon::logger();
} }
bool isCancelled() const bool isCancelled() const override
{ {
return BaseDaemon::isCancelled(); return BaseDaemon::isCancelled();
} }
/// Returns global application's context.
ContextMutablePtr context() const override
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot fetch context for Keeper");
}
void defineOptions(Poco::Util::OptionSet & _options) override; void defineOptions(Poco::Util::OptionSet & _options) override;
protected: protected:
@ -56,7 +68,10 @@ protected:
std::string getDefaultConfigFileName() const override; std::string getDefaultConfigFileName() const override;
private: private:
TinyContext tiny_context; TinyContextPtr tiny_context;
struct KeeperHTTPContext;
HTTPContextPtr httpContext();
Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const; Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const;

View File

@ -1,6 +1,6 @@
#include "MetricsTransmitter.h" #include "MetricsTransmitter.h"
#include <Interpreters/AsynchronousMetrics.h> #include <Common/AsynchronousMetrics.h>
#include <Common/CurrentMetrics.h> #include <Common/CurrentMetrics.h>
#include <Common/Exception.h> #include <Common/Exception.h>

View File

@ -46,7 +46,7 @@
#include <IO/ReadBufferFromFile.h> #include <IO/ReadBufferFromFile.h>
#include <IO/IOThreadPool.h> #include <IO/IOThreadPool.h>
#include <IO/UseSSL.h> #include <IO/UseSSL.h>
#include <Interpreters/AsynchronousMetrics.h> #include <Interpreters/ServerAsynchronousMetrics.h>
#include <Interpreters/DDLWorker.h> #include <Interpreters/DDLWorker.h>
#include <Interpreters/DNSCacheUpdater.h> #include <Interpreters/DNSCacheUpdater.h>
#include <Interpreters/DatabaseCatalog.h> #include <Interpreters/DatabaseCatalog.h>
@ -803,7 +803,7 @@ try
std::vector<ProtocolServerAdapter> servers; std::vector<ProtocolServerAdapter> servers;
std::vector<ProtocolServerAdapter> servers_to_start_before_tables; std::vector<ProtocolServerAdapter> servers_to_start_before_tables;
/// This object will periodically calculate some metrics. /// This object will periodically calculate some metrics.
AsynchronousMetrics async_metrics( ServerAsynchronousMetrics async_metrics(
global_context, global_context,
config().getUInt("asynchronous_metrics_update_period_s", 1), config().getUInt("asynchronous_metrics_update_period_s", 1),
config().getUInt("asynchronous_heavy_metrics_update_period_s", 120), config().getUInt("asynchronous_heavy_metrics_update_period_s", 120),
@ -1947,15 +1947,15 @@ std::unique_ptr<TCPProtocolStackFactory> Server::buildProtocolStackFromConfig(
return TCPServerConnectionFactory::Ptr(new PostgreSQLHandlerFactory(*this)); return TCPServerConnectionFactory::Ptr(new PostgreSQLHandlerFactory(*this));
if (type == "http") if (type == "http")
return TCPServerConnectionFactory::Ptr( return TCPServerConnectionFactory::Ptr(
new HTTPServerConnectionFactory(context(), http_params, createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory")) new HTTPServerConnectionFactory(httpContext(), http_params, createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory"))
); );
if (type == "prometheus") if (type == "prometheus")
return TCPServerConnectionFactory::Ptr( return TCPServerConnectionFactory::Ptr(
new HTTPServerConnectionFactory(context(), http_params, createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory")) new HTTPServerConnectionFactory(httpContext(), http_params, createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"))
); );
if (type == "interserver") if (type == "interserver")
return TCPServerConnectionFactory::Ptr( return TCPServerConnectionFactory::Ptr(
new HTTPServerConnectionFactory(context(), http_params, createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory")) new HTTPServerConnectionFactory(httpContext(), http_params, createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory"))
); );
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol configuration error, unknown protocol name '{}'", type); throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol configuration error, unknown protocol name '{}'", type);
@ -1996,6 +1996,11 @@ std::unique_ptr<TCPProtocolStackFactory> Server::buildProtocolStackFromConfig(
return stack; return stack;
} }
HTTPContextPtr Server::httpContext() const
{
return std::make_shared<HTTPContext>(context());
}
void Server::createServers( void Server::createServers(
Poco::Util::AbstractConfiguration & config, Poco::Util::AbstractConfiguration & config,
const Strings & listen_hosts, const Strings & listen_hosts,
@ -2078,7 +2083,7 @@ void Server::createServers(
port_name, port_name,
"http://" + address.toString(), "http://" + address.toString(),
std::make_unique<HTTPServer>( std::make_unique<HTTPServer>(
context(), createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params)); httpContext(), createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params));
}); });
/// HTTPS /// HTTPS
@ -2095,7 +2100,7 @@ void Server::createServers(
port_name, port_name,
"https://" + address.toString(), "https://" + address.toString(),
std::make_unique<HTTPServer>( std::make_unique<HTTPServer>(
context(), createHandlerFactory(*this, config, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params)); httpContext(), createHandlerFactory(*this, config, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params));
#else #else
UNUSED(port); UNUSED(port);
throw Exception{"HTTPS protocol is disabled because Poco library was built without NetSSL support.", throw Exception{"HTTPS protocol is disabled because Poco library was built without NetSSL support.",
@ -2220,7 +2225,7 @@ void Server::createServers(
port_name, port_name,
"Prometheus: http://" + address.toString(), "Prometheus: http://" + address.toString(),
std::make_unique<HTTPServer>( std::make_unique<HTTPServer>(
context(), createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); httpContext(), createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params));
}); });
} }
@ -2240,7 +2245,7 @@ void Server::createServers(
port_name, port_name,
"replica communication (interserver): http://" + address.toString(), "replica communication (interserver): http://" + address.toString(),
std::make_unique<HTTPServer>( std::make_unique<HTTPServer>(
context(), httpContext(),
createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory"), createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory"),
server_pool, server_pool,
socket, socket,
@ -2260,7 +2265,7 @@ void Server::createServers(
port_name, port_name,
"secure replica communication (interserver): https://" + address.toString(), "secure replica communication (interserver): https://" + address.toString(),
std::make_unique<HTTPServer>( std::make_unique<HTTPServer>(
context(), httpContext(),
createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPSHandler-factory"), createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPSHandler-factory"),
server_pool, server_pool,
socket, socket,

View File

@ -3,6 +3,7 @@
#include <Server/IServer.h> #include <Server/IServer.h>
#include <Daemon/BaseDaemon.h> #include <Daemon/BaseDaemon.h>
#include "Server/HTTP/HTTPContext.h"
#include <Server/TCPProtocolStackFactory.h> #include <Server/TCPProtocolStackFactory.h>
#include <Poco/Net/HTTPServerParams.h> #include <Poco/Net/HTTPServerParams.h>
@ -72,6 +73,8 @@ private:
/// Updated/recent config, to compare http_handlers /// Updated/recent config, to compare http_handlers
ConfigurationPtr latest_config; ConfigurationPtr latest_config;
HTTPContextPtr httpContext() const;
Poco::Net::SocketAddress socketBindListen( Poco::Net::SocketAddress socketBindListen(
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
Poco::Net::ServerSocket & socket, Poco::Net::ServerSocket & socket,

View File

@ -160,7 +160,7 @@ public:
else else
{ {
writeBinary(UInt8(0), buf); writeBinary(UInt8(0), buf);
serialization->serializeBinary(elem, buf); serialization->serializeBinary(elem, buf, {});
} }
} }
} }
@ -181,7 +181,7 @@ public:
UInt8 is_null = 0; UInt8 is_null = 0;
readBinary(is_null, buf); readBinary(is_null, buf);
if (!is_null) if (!is_null)
serialization->deserializeBinary(arr[i], buf); serialization->deserializeBinary(arr[i], buf, {});
} }
} }

View File

@ -795,7 +795,7 @@ public:
if (!value.isNull()) if (!value.isNull())
{ {
writeBinary(true, buf); writeBinary(true, buf);
serialization.serializeBinary(value, buf); serialization.serializeBinary(value, buf, {});
} }
else else
writeBinary(false, buf); writeBinary(false, buf);
@ -807,7 +807,7 @@ public:
readBinary(is_not_null, buf); readBinary(is_not_null, buf);
if (is_not_null) if (is_not_null)
serialization.deserializeBinary(value, buf); serialization.deserializeBinary(value, buf, {});
} }
void change(const IColumn & column, size_t row_num, Arena *) void change(const IColumn & column, size_t row_num, Arena *)

View File

@ -296,19 +296,19 @@ public:
{ {
case 0: case 0:
{ {
serialize = [&](size_t col_idx, const Array & values){ values_serializations[col_idx]->serializeBinary(values[col_idx], buf); }; serialize = [&](size_t col_idx, const Array & values){ values_serializations[col_idx]->serializeBinary(values[col_idx], buf, {}); };
break; break;
} }
case 1: case 1:
{ {
serialize = [&](size_t col_idx, const Array & values){ promoted_values_serializations[col_idx]->serializeBinary(values[col_idx], buf); }; serialize = [&](size_t col_idx, const Array & values){ promoted_values_serializations[col_idx]->serializeBinary(values[col_idx], buf, {}); };
break; break;
} }
} }
for (const auto & elem : merged_maps) for (const auto & elem : merged_maps)
{ {
keys_serialization->serializeBinary(elem.first, buf); keys_serialization->serializeBinary(elem.first, buf, {});
for (size_t col = 0; col < values_types.size(); ++col) for (size_t col = 0; col < values_types.size(); ++col)
serialize(col, elem.second); serialize(col, elem.second);
} }
@ -328,12 +328,12 @@ public:
{ {
case 0: case 0:
{ {
deserialize = [&](size_t col_idx, Array & values){ values_serializations[col_idx]->deserializeBinary(values[col_idx], buf); }; deserialize = [&](size_t col_idx, Array & values){ values_serializations[col_idx]->deserializeBinary(values[col_idx], buf, {}); };
break; break;
} }
case 1: case 1:
{ {
deserialize = [&](size_t col_idx, Array & values){ promoted_values_serializations[col_idx]->deserializeBinary(values[col_idx], buf); }; deserialize = [&](size_t col_idx, Array & values){ promoted_values_serializations[col_idx]->deserializeBinary(values[col_idx], buf, {}); };
break; break;
} }
} }
@ -341,7 +341,7 @@ public:
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
{ {
Field key; Field key;
keys_serialization->deserializeBinary(key, buf); keys_serialization->deserializeBinary(key, buf, {});
Array values; Array values;
values.resize(values_types.size()); values.resize(values_types.size());

View File

@ -79,8 +79,6 @@ FunctionNodePtr createResolvedAggregateFunction(const String & name, const Query
function_node->resolveAsAggregateFunction(aggregate_function, aggregate_function->getReturnType()); function_node->resolveAsAggregateFunction(aggregate_function, aggregate_function->getReturnType());
function_node->getArguments().getNodes() = { argument }; function_node->getArguments().getNodes() = { argument };
function_node->getArguments().getNodes() = { argument };
if (!parameters.empty()) if (!parameters.empty())
{ {
QueryTreeNodes parameter_nodes; QueryTreeNodes parameter_nodes;

View File

@ -166,7 +166,8 @@ void BackupWriterS3::copyObjectImpl(
auto outcome = client->CopyObject(request); auto outcome = client->CopyObject(request);
if (!outcome.IsSuccess() && outcome.GetError().GetExceptionName() == "EntityTooLarge") if (!outcome.IsSuccess() && (outcome.GetError().GetExceptionName() == "EntityTooLarge"
|| outcome.GetError().GetExceptionName() == "InvalidRequest"))
{ // Can't come here with MinIO, MinIO allows single part upload for large objects. { // Can't come here with MinIO, MinIO allows single part upload for large objects.
copyObjectMultipartImpl(src_bucket, src_key, dst_bucket, dst_key, head, metadata); copyObjectMultipartImpl(src_bucket, src_key, dst_bucket, dst_key, head, metadata);
return; return;

View File

@ -96,6 +96,7 @@ RestorerFromBackup::RestorerFromBackup(
, on_cluster_first_sync_timeout(context->getConfigRef().getUInt64("backups.on_cluster_first_sync_timeout", 180000)) , on_cluster_first_sync_timeout(context->getConfigRef().getUInt64("backups.on_cluster_first_sync_timeout", 180000))
, create_table_timeout(context->getConfigRef().getUInt64("backups.create_table_timeout", 300000)) , create_table_timeout(context->getConfigRef().getUInt64("backups.create_table_timeout", 300000))
, log(&Poco::Logger::get("RestorerFromBackup")) , log(&Poco::Logger::get("RestorerFromBackup"))
, tables_dependencies("RestorerFromBackup")
{ {
} }
@ -133,6 +134,7 @@ RestorerFromBackup::DataRestoreTasks RestorerFromBackup::run(Mode mode)
/// Create tables using the create queries read from the backup. /// Create tables using the create queries read from the backup.
setStage(Stage::CREATING_TABLES); setStage(Stage::CREATING_TABLES);
removeUnresolvedDependencies();
createTables(); createTables();
/// All what's left is to insert data to tables. /// All what's left is to insert data to tables.
@ -341,10 +343,11 @@ void RestorerFromBackup::findTableInBackup(const QualifiedTableName & table_name
TableInfo & res_table_info = table_infos[table_name]; TableInfo & res_table_info = table_infos[table_name];
res_table_info.create_table_query = create_table_query; res_table_info.create_table_query = create_table_query;
res_table_info.is_predefined_table = DatabaseCatalog::instance().isPredefinedTable(StorageID{table_name.database, table_name.table}); res_table_info.is_predefined_table = DatabaseCatalog::instance().isPredefinedTable(StorageID{table_name.database, table_name.table});
res_table_info.dependencies = getDependenciesSetFromCreateQuery(context->getGlobalContext(), table_name, create_table_query);
res_table_info.has_data = backup->hasFiles(data_path_in_backup); res_table_info.has_data = backup->hasFiles(data_path_in_backup);
res_table_info.data_path_in_backup = data_path_in_backup; res_table_info.data_path_in_backup = data_path_in_backup;
tables_dependencies.addDependencies(table_name, getDependenciesFromCreateQuery(context->getGlobalContext(), table_name, create_table_query));
if (partitions) if (partitions)
{ {
if (!res_table_info.partitions) if (!res_table_info.partitions)
@ -622,23 +625,64 @@ void RestorerFromBackup::checkDatabase(const String & database_name)
} }
} }
void RestorerFromBackup::removeUnresolvedDependencies()
{
auto need_exclude_dependency = [this](const StorageID & table_id)
{
/// Table will be restored.
if (table_infos.contains(table_id.getQualifiedName()))
return false;
/// Table exists and it already exists
if (!DatabaseCatalog::instance().isTableExist(table_id, context))
{
LOG_WARNING(
log,
"Tables {} in backup depend on {}, but seems like {} is not in the backup and does not exist. "
"Will try to ignore that and restore tables",
fmt::join(tables_dependencies.getDependents(table_id), ", "),
table_id,
table_id);
}
size_t num_dependencies, num_dependents;
tables_dependencies.getNumberOfAdjacents(table_id, num_dependencies, num_dependents);
if (num_dependencies || !num_dependents)
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Table {} in backup doesn't have dependencies and dependent tables as it expected to. It's a bug",
table_id);
return true; /// Exclude this dependency.
};
tables_dependencies.removeTablesIf(need_exclude_dependency);
if (tables_dependencies.getNumberOfTables() != table_infos.size())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Number of tables to be restored is not as expected. It's a bug");
if (tables_dependencies.hasCyclicDependencies())
{
LOG_WARNING(
log,
"Tables {} in backup have cyclic dependencies: {}. Will try to ignore that and restore tables",
fmt::join(tables_dependencies.getTablesWithCyclicDependencies(), ", "),
tables_dependencies.describeCyclicDependencies());
}
}
void RestorerFromBackup::createTables() void RestorerFromBackup::createTables()
{
while (true)
{ {
/// We need to create tables considering their dependencies. /// We need to create tables considering their dependencies.
auto tables_to_create = findTablesWithoutDependencies(); auto tables_to_create = tables_dependencies.getTablesSortedByDependency();
if (tables_to_create.empty()) for (const auto & table_id : tables_to_create)
break; /// We've already created all the tables.
for (const auto & table_name : tables_to_create)
{ {
auto table_name = table_id.getQualifiedName();
createTable(table_name); createTable(table_name);
checkTable(table_name); checkTable(table_name);
insertDataToTable(table_name); insertDataToTable(table_name);
} }
} }
}
void RestorerFromBackup::createTable(const QualifiedTableName & table_name) void RestorerFromBackup::createTable(const QualifiedTableName & table_name)
{ {
@ -752,62 +796,6 @@ void RestorerFromBackup::insertDataToTable(const QualifiedTableName & table_name
} }
} }
/// Returns the list of tables without dependencies or those which dependencies have been created before.
std::vector<QualifiedTableName> RestorerFromBackup::findTablesWithoutDependencies() const
{
std::vector<QualifiedTableName> tables_without_dependencies;
bool all_tables_created = true;
for (const auto & [key, table_info] : table_infos)
{
if (table_info.storage)
continue;
/// Found a table which is not created yet.
all_tables_created = false;
/// Check if all dependencies have been created before.
bool all_dependencies_met = true;
for (const auto & dependency : table_info.dependencies)
{
auto it = table_infos.find(dependency);
if ((it != table_infos.end()) && !it->second.storage)
{
all_dependencies_met = false;
break;
}
}
if (all_dependencies_met)
tables_without_dependencies.push_back(key);
}
if (!tables_without_dependencies.empty())
return tables_without_dependencies;
if (all_tables_created)
return {};
/// Cyclic dependency? We'll try to create those tables anyway but probably it's going to fail.
std::vector<QualifiedTableName> tables_with_cyclic_dependencies;
for (const auto & [key, table_info] : table_infos)
{
if (!table_info.storage)
tables_with_cyclic_dependencies.push_back(key);
}
/// Only show a warning here, proper exception will be thrown later on creating those tables.
LOG_WARNING(
log,
"Some tables have cyclic dependency from each other: {}",
boost::algorithm::join(
tables_with_cyclic_dependencies
| boost::adaptors::transformed([](const QualifiedTableName & table_name) -> String { return table_name.getFullName(); }),
", "));
return tables_with_cyclic_dependencies;
}
void RestorerFromBackup::addDataRestoreTask(DataRestoreTask && new_task) void RestorerFromBackup::addDataRestoreTask(DataRestoreTask && new_task)
{ {
if (current_stage == Stage::INSERTING_DATA_TO_TABLES) if (current_stage == Stage::INSERTING_DATA_TO_TABLES)

View File

@ -2,6 +2,7 @@
#include <Backups/RestoreSettings.h> #include <Backups/RestoreSettings.h>
#include <Databases/DDLRenamingVisitor.h> #include <Databases/DDLRenamingVisitor.h>
#include <Databases/TablesDependencyGraph.h>
#include <Parsers/ASTBackupQuery.h> #include <Parsers/ASTBackupQuery.h>
#include <Storages/TableLockHolder.h> #include <Storages/TableLockHolder.h>
#include <Storages/IStorage_fwd.h> #include <Storages/IStorage_fwd.h>
@ -94,6 +95,7 @@ private:
void createDatabase(const String & database_name) const; void createDatabase(const String & database_name) const;
void checkDatabase(const String & database_name); void checkDatabase(const String & database_name);
void removeUnresolvedDependencies();
void createTables(); void createTables();
void createTable(const QualifiedTableName & table_name); void createTable(const QualifiedTableName & table_name);
void checkTable(const QualifiedTableName & table_name); void checkTable(const QualifiedTableName & table_name);
@ -114,7 +116,6 @@ private:
{ {
ASTPtr create_table_query; ASTPtr create_table_query;
bool is_predefined_table = false; bool is_predefined_table = false;
std::unordered_set<QualifiedTableName> dependencies;
bool has_data = false; bool has_data = false;
std::filesystem::path data_path_in_backup; std::filesystem::path data_path_in_backup;
std::optional<ASTs> partitions; std::optional<ASTs> partitions;
@ -123,11 +124,10 @@ private:
TableLockHolder table_lock; TableLockHolder table_lock;
}; };
std::vector<QualifiedTableName> findTablesWithoutDependencies() const;
String current_stage; String current_stage;
std::unordered_map<String, DatabaseInfo> database_infos; std::unordered_map<String, DatabaseInfo> database_infos;
std::map<QualifiedTableName, TableInfo> table_infos; std::map<QualifiedTableName, TableInfo> table_infos;
TablesDependencyGraph tables_dependencies;
std::vector<DataRestoreTask> data_restore_tasks; std::vector<DataRestoreTask> data_restore_tasks;
std::unique_ptr<AccessRestorerFromBackup> access_restorer; std::unique_ptr<AccessRestorerFromBackup> access_restorer;
bool access_restored = false; bool access_restored = false;

View File

@ -236,7 +236,7 @@ int IBridge::main(const std::vector<std::string> & /*args*/)
SensitiveDataMasker::setInstance(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules")); SensitiveDataMasker::setInstance(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
auto server = HTTPServer( auto server = HTTPServer(
context, std::make_shared<HTTPContext>(context),
getHandlerFactoryPtr(context), getHandlerFactoryPtr(context),
server_pool, server_pool,
socket, socket,

View File

@ -148,7 +148,8 @@ void Connection::connect(const ConnectionTimeouts & timeouts)
socket->setReceiveTimeout(timeouts.receive_timeout); socket->setReceiveTimeout(timeouts.receive_timeout);
socket->setSendTimeout(timeouts.send_timeout); socket->setSendTimeout(timeouts.send_timeout);
socket->setNoDelay(true); socket->setNoDelay(true);
if (timeouts.tcp_keep_alive_timeout.totalSeconds()) int tcp_keep_alive_timeout_in_sec = timeouts.tcp_keep_alive_timeout.totalSeconds();
if (tcp_keep_alive_timeout_in_sec)
{ {
socket->setKeepAlive(true); socket->setKeepAlive(true);
socket->setOption(IPPROTO_TCP, socket->setOption(IPPROTO_TCP,
@ -157,7 +158,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts)
#else #else
TCP_KEEPIDLE // __APPLE__ TCP_KEEPIDLE // __APPLE__
#endif #endif
, timeouts.tcp_keep_alive_timeout); , tcp_keep_alive_timeout_in_sec);
} }
in = std::make_shared<ReadBufferFromPocoSocket>(*socket); in = std::make_shared<ReadBufferFromPocoSocket>(*socket);

View File

@ -1,28 +1,16 @@
#include <Interpreters/Aggregator.h> #include <Common/AsynchronousMetrics.h>
#include <Interpreters/AsynchronousMetrics.h>
#include <Interpreters/AsynchronousMetricLog.h>
#include <Interpreters/JIT/CompiledExpressionCache.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/Context.h>
#include <Coordination/Keeper4LWInfo.h>
#include <Coordination/KeeperDispatcher.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/setThreadName.h> #include <Common/setThreadName.h>
#include <Common/CurrentMetrics.h> #include <Common/CurrentMetrics.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Common/filesystemHelpers.h> #include <Common/filesystemHelpers.h>
#include <Interpreters/Cache/FileCacheFactory.h>
#include <Common/getCurrentProcessFDCount.h> #include <Common/getCurrentProcessFDCount.h>
#include <Common/getMaxFileDescriptorCount.h> #include <Common/getMaxFileDescriptorCount.h>
#include <Interpreters/Cache/FileCache.h> #include <Interpreters/Cache/FileCache.h>
#include <Storages/MarkCache.h> #include <Server/ProtocolServerAdapter.h>
#include <Storages/StorageMergeTree.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/MergeTree/MergeTreeMetadataCache.h>
#include <IO/UncompressedCache.h> #include <IO/UncompressedCache.h>
#include <IO/MMappedFileCache.h> #include <IO/MMappedFileCache.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <Databases/IDatabase.h>
#include <base/errnoToString.h> #include <base/errnoToString.h>
#include <chrono> #include <chrono>
@ -68,15 +56,11 @@ static std::unique_ptr<ReadBufferFromFilePRead> openFileIfExists(const std::stri
AsynchronousMetrics::AsynchronousMetrics( AsynchronousMetrics::AsynchronousMetrics(
ContextPtr global_context_,
int update_period_seconds, int update_period_seconds,
int heavy_metrics_update_period_seconds,
const ProtocolServerMetricsFunc & protocol_server_metrics_func_) const ProtocolServerMetricsFunc & protocol_server_metrics_func_)
: WithContext(global_context_) : update_period(update_period_seconds)
, update_period(update_period_seconds)
, heavy_metric_update_period(heavy_metrics_update_period_seconds)
, protocol_server_metrics_func(protocol_server_metrics_func_)
, log(&Poco::Logger::get("AsynchronousMetrics")) , log(&Poco::Logger::get("AsynchronousMetrics"))
, protocol_server_metrics_func(protocol_server_metrics_func_)
{ {
#if defined(OS_LINUX) #if defined(OS_LINUX)
openFileIfExists("/proc/meminfo", meminfo); openFileIfExists("/proc/meminfo", meminfo);
@ -360,22 +344,6 @@ void AsynchronousMetrics::run()
} }
} }
template <typename Max, typename T>
static void calculateMax(Max & max, T x)
{
if (Max(x) > max)
max = x;
}
template <typename Max, typename Sum, typename T>
static void calculateMaxAndSum(Max & max, Sum & sum, T x)
{
sum += x;
if (Max(x) > max)
max = x;
}
#if USE_JEMALLOC #if USE_JEMALLOC
uint64_t updateJemallocEpoch() uint64_t updateJemallocEpoch()
{ {
@ -575,91 +543,6 @@ void AsynchronousMetrics::update(TimePoint update_time)
"The difference in time the thread for calculation of the asynchronous metrics was scheduled to wake up and the time it was in fact, woken up." "The difference in time the thread for calculation of the asynchronous metrics was scheduled to wake up and the time it was in fact, woken up."
" A proxy-indicator of overall system latency and responsiveness." }; " A proxy-indicator of overall system latency and responsiveness." };
if (auto mark_cache = getContext()->getMarkCache())
{
new_values["MarkCacheBytes"] = { mark_cache->weight(), "Total size of mark cache in bytes" };
new_values["MarkCacheFiles"] = { mark_cache->count(), "Total number of mark files cached in the mark cache" };
}
if (auto uncompressed_cache = getContext()->getUncompressedCache())
{
new_values["UncompressedCacheBytes"] = { uncompressed_cache->weight(),
"Total size of uncompressed cache in bytes. Uncompressed cache does not usually improve the performance and should be mostly avoided." };
new_values["UncompressedCacheCells"] = { uncompressed_cache->count(),
"Total number of entries in the uncompressed cache. Each entry represents a decompressed block of data. Uncompressed cache does not usually improve performance and should be mostly avoided." };
}
if (auto index_mark_cache = getContext()->getIndexMarkCache())
{
new_values["IndexMarkCacheBytes"] = { index_mark_cache->weight(), "Total size of mark cache for secondary indices in bytes." };
new_values["IndexMarkCacheFiles"] = { index_mark_cache->count(), "Total number of mark files cached in the mark cache for secondary indices." };
}
if (auto index_uncompressed_cache = getContext()->getIndexUncompressedCache())
{
new_values["IndexUncompressedCacheBytes"] = { index_uncompressed_cache->weight(),
"Total size of uncompressed cache in bytes for secondary indices. Uncompressed cache does not usually improve the performance and should be mostly avoided." };
new_values["IndexUncompressedCacheCells"] = { index_uncompressed_cache->count(),
"Total number of entries in the uncompressed cache for secondary indices. Each entry represents a decompressed block of data. Uncompressed cache does not usually improve performance and should be mostly avoided." };
}
if (auto mmap_cache = getContext()->getMMappedFileCache())
{
new_values["MMapCacheCells"] = { mmap_cache->count(),
"The number of files opened with `mmap` (mapped in memory)."
" This is used for queries with the setting `local_filesystem_read_method` set to `mmap`."
" The files opened with `mmap` are kept in the cache to avoid costly TLB flushes."};
}
{
auto caches = FileCacheFactory::instance().getAll();
size_t total_bytes = 0;
size_t total_files = 0;
for (const auto & [_, cache_data] : caches)
{
total_bytes += cache_data->cache->getUsedCacheSize();
total_files += cache_data->cache->getFileSegmentsNum();
}
new_values["FilesystemCacheBytes"] = { total_bytes,
"Total bytes in the `cache` virtual filesystem. This cache is hold on disk." };
new_values["FilesystemCacheFiles"] = { total_files,
"Total number of cached file segments in the `cache` virtual filesystem. This cache is hold on disk." };
}
#if USE_ROCKSDB
if (auto metadata_cache = getContext()->tryGetMergeTreeMetadataCache())
{
new_values["MergeTreeMetadataCacheSize"] = { metadata_cache->getEstimateNumKeys(),
"The size of the metadata cache for tables. This cache is experimental and not used in production." };
}
#endif
#if USE_EMBEDDED_COMPILER
if (auto * compiled_expression_cache = CompiledExpressionCacheFactory::instance().tryGetCache())
{
new_values["CompiledExpressionCacheBytes"] = { compiled_expression_cache->weight(),
"Total bytes used for the cache of JIT-compiled code." };
new_values["CompiledExpressionCacheCount"] = { compiled_expression_cache->count(),
"Total entries in the cache of JIT-compiled code." };
}
#endif
new_values["Uptime"] = { getContext()->getUptimeSeconds(),
"The server uptime in seconds. It includes the time spent for server initialization before accepting connections." };
if (const auto stats = getHashTablesCacheStatistics())
{
new_values["HashTableStatsCacheEntries"] = { stats->entries,
"The number of entries in the cache of hash table sizes."
" The cache for hash table sizes is used for predictive optimization of GROUP BY." };
new_values["HashTableStatsCacheHits"] = { stats->hits,
"The number of times the prediction of a hash table size was correct." };
new_values["HashTableStatsCacheMisses"] = { stats->misses,
"The number of times the prediction of a hash table size was incorrect." };
}
#if defined(OS_LINUX) || defined(OS_FREEBSD) #if defined(OS_LINUX) || defined(OS_FREEBSD)
MemoryStatisticsOS::Data memory_statistics_data = memory_stat.get(); MemoryStatisticsOS::Data memory_statistics_data = memory_stat.get();
#endif #endif
@ -1519,165 +1402,7 @@ void AsynchronousMetrics::update(TimePoint update_time)
} }
#endif #endif
/// Free space in filesystems at data path and logs path.
{ {
auto stat = getStatVFS(getContext()->getPath());
new_values["FilesystemMainPathTotalBytes"] = { stat.f_blocks * stat.f_frsize,
"The size of the volume where the main ClickHouse path is mounted, in bytes." };
new_values["FilesystemMainPathAvailableBytes"] = { stat.f_bavail * stat.f_frsize,
"Available bytes on the volume where the main ClickHouse path is mounted." };
new_values["FilesystemMainPathUsedBytes"] = { (stat.f_blocks - stat.f_bavail) * stat.f_frsize,
"Used bytes on the volume where the main ClickHouse path is mounted." };
new_values["FilesystemMainPathTotalINodes"] = { stat.f_files,
"The total number of inodes on the volume where the main ClickHouse path is mounted. If it is less than 25 million, it indicates a misconfiguration." };
new_values["FilesystemMainPathAvailableINodes"] = { stat.f_favail,
"The number of available inodes on the volume where the main ClickHouse path is mounted. If it is close to zero, it indicates a misconfiguration, and you will get 'no space left on device' even when the disk is not full." };
new_values["FilesystemMainPathUsedINodes"] = { stat.f_files - stat.f_favail,
"The number of used inodes on the volume where the main ClickHouse path is mounted. This value mostly corresponds to the number of files." };
}
{
/// Current working directory of the server is the directory with logs.
auto stat = getStatVFS(".");
new_values["FilesystemLogsPathTotalBytes"] = { stat.f_blocks * stat.f_frsize,
"The size of the volume where ClickHouse logs path is mounted, in bytes. It's recommended to have at least 10 GB for logs." };
new_values["FilesystemLogsPathAvailableBytes"] = { stat.f_bavail * stat.f_frsize,
"Available bytes on the volume where ClickHouse logs path is mounted. If this value approaches zero, you should tune the log rotation in the configuration file." };
new_values["FilesystemLogsPathUsedBytes"] = { (stat.f_blocks - stat.f_bavail) * stat.f_frsize,
"Used bytes on the volume where ClickHouse logs path is mounted." };
new_values["FilesystemLogsPathTotalINodes"] = { stat.f_files,
"The total number of inodes on the volume where ClickHouse logs path is mounted." };
new_values["FilesystemLogsPathAvailableINodes"] = { stat.f_favail,
"The number of available inodes on the volume where ClickHouse logs path is mounted." };
new_values["FilesystemLogsPathUsedINodes"] = { stat.f_files - stat.f_favail,
"The number of used inodes on the volume where ClickHouse logs path is mounted." };
}
/// Free and total space on every configured disk.
{
DisksMap disks_map = getContext()->getDisksMap();
for (const auto & [name, disk] : disks_map)
{
auto total = disk->getTotalSpace();
/// Some disks don't support information about the space.
if (!total)
continue;
auto available = disk->getAvailableSpace();
auto unreserved = disk->getUnreservedSpace();
new_values[fmt::format("DiskTotal_{}", name)] = { total,
"The total size in bytes of the disk (virtual filesystem). Remote filesystems can show a large value like 16 EiB." };
new_values[fmt::format("DiskUsed_{}", name)] = { total - available,
"Used bytes on the disk (virtual filesystem). Remote filesystems not always provide this information." };
new_values[fmt::format("DiskAvailable_{}", name)] = { available,
"Available bytes on the disk (virtual filesystem). Remote filesystems can show a large value like 16 EiB." };
new_values[fmt::format("DiskUnreserved_{}", name)] = { unreserved,
"Available bytes on the disk (virtual filesystem) without the reservations for merges, fetches, and moves. Remote filesystems can show a large value like 16 EiB." };
}
}
{
auto databases = DatabaseCatalog::instance().getDatabases();
size_t max_queue_size = 0;
size_t max_inserts_in_queue = 0;
size_t max_merges_in_queue = 0;
size_t sum_queue_size = 0;
size_t sum_inserts_in_queue = 0;
size_t sum_merges_in_queue = 0;
size_t max_absolute_delay = 0;
size_t max_relative_delay = 0;
size_t max_part_count_for_partition = 0;
size_t number_of_databases = databases.size();
size_t total_number_of_tables = 0;
size_t total_number_of_bytes = 0;
size_t total_number_of_rows = 0;
size_t total_number_of_parts = 0;
for (const auto & db : databases)
{
/// Check if database can contain MergeTree tables
if (!db.second->canContainMergeTreeTables())
continue;
for (auto iterator = db.second->getTablesIterator(getContext()); iterator->isValid(); iterator->next())
{
++total_number_of_tables;
const auto & table = iterator->table();
if (!table)
continue;
if (MergeTreeData * table_merge_tree = dynamic_cast<MergeTreeData *>(table.get()))
{
const auto & settings = getContext()->getSettingsRef();
calculateMax(max_part_count_for_partition, table_merge_tree->getMaxPartsCountAndSizeForPartition().first);
total_number_of_bytes += table_merge_tree->totalBytes(settings).value();
total_number_of_rows += table_merge_tree->totalRows(settings).value();
total_number_of_parts += table_merge_tree->getPartsCount();
}
if (StorageReplicatedMergeTree * table_replicated_merge_tree = typeid_cast<StorageReplicatedMergeTree *>(table.get()))
{
StorageReplicatedMergeTree::Status status;
table_replicated_merge_tree->getStatus(status, false);
calculateMaxAndSum(max_queue_size, sum_queue_size, status.queue.queue_size);
calculateMaxAndSum(max_inserts_in_queue, sum_inserts_in_queue, status.queue.inserts_in_queue);
calculateMaxAndSum(max_merges_in_queue, sum_merges_in_queue, status.queue.merges_in_queue);
if (!status.is_readonly)
{
try
{
time_t absolute_delay = 0;
time_t relative_delay = 0;
table_replicated_merge_tree->getReplicaDelays(absolute_delay, relative_delay);
calculateMax(max_absolute_delay, absolute_delay);
calculateMax(max_relative_delay, relative_delay);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__,
"Cannot get replica delay for table: " + backQuoteIfNeed(db.first) + "." + backQuoteIfNeed(iterator->name()));
}
}
}
}
}
new_values["ReplicasMaxQueueSize"] = { max_queue_size, "Maximum queue size (in the number of operations like get, merge) across Replicated tables." };
new_values["ReplicasMaxInsertsInQueue"] = { max_inserts_in_queue, "Maximum number of INSERT operations in the queue (still to be replicated) across Replicated tables." };
new_values["ReplicasMaxMergesInQueue"] = { max_merges_in_queue, "Maximum number of merge operations in the queue (still to be applied) across Replicated tables." };
new_values["ReplicasSumQueueSize"] = { sum_queue_size, "Sum queue size (in the number of operations like get, merge) across Replicated tables." };
new_values["ReplicasSumInsertsInQueue"] = { sum_inserts_in_queue, "Sum of INSERT operations in the queue (still to be replicated) across Replicated tables." };
new_values["ReplicasSumMergesInQueue"] = { sum_merges_in_queue, "Sum of merge operations in the queue (still to be applied) across Replicated tables." };
new_values["ReplicasMaxAbsoluteDelay"] = { max_absolute_delay, "Maximum difference in seconds between the most fresh replicated part and the most fresh data part still to be replicated, across Replicated tables. A very high value indicates a replica with no data." };
new_values["ReplicasMaxRelativeDelay"] = { max_relative_delay, "Maximum difference between the replica delay and the delay of the most up-to-date replica of the same table, across Replicated tables." };
new_values["MaxPartCountForPartition"] = { max_part_count_for_partition, "Maximum number of parts per partition across all partitions of all tables of MergeTree family. Values larger than 300 indicates misconfiguration, overload, or massive data loading." };
new_values["NumberOfDatabases"] = { number_of_databases, "Total number of databases on the server." };
new_values["NumberOfTables"] = { total_number_of_tables, "Total number of tables summed across the databases on the server, excluding the databases that cannot contain MergeTree tables."
" The excluded database engines are those who generate the set of tables on the fly, like `Lazy`, `MySQL`, `PostgreSQL`, `SQlite`."};
new_values["TotalBytesOfMergeTreeTables"] = { total_number_of_bytes, "Total amount of bytes (compressed, including data and indices) stored in all tables of MergeTree family." };
new_values["TotalRowsOfMergeTreeTables"] = { total_number_of_rows, "Total amount of rows (records) stored in all tables of MergeTree family." };
new_values["TotalPartsOfMergeTreeTables"] = { total_number_of_parts, "Total amount of data parts in all tables of MergeTree family."
" Numbers larger than 10 000 will negatively affect the server startup time and it may indicate unreasonable choice of the partition key." };
auto get_metric_name_doc = [](const String & name) -> std::pair<const char *, const char *> auto get_metric_name_doc = [](const String & name) -> std::pair<const char *, const char *>
{ {
static std::map<String, std::pair<const char *, const char *>> metric_map = static std::map<String, std::pair<const char *, const char *>> metric_map =
@ -1691,7 +1416,9 @@ void AsynchronousMetrics::update(TimePoint update_time)
{"mysql_port", {"MySQLThreads", "Number of threads in the server of the MySQL compatibility protocol."}}, {"mysql_port", {"MySQLThreads", "Number of threads in the server of the MySQL compatibility protocol."}},
{"postgresql_port", {"PostgreSQLThreads", "Number of threads in the server of the PostgreSQL compatibility protocol."}}, {"postgresql_port", {"PostgreSQLThreads", "Number of threads in the server of the PostgreSQL compatibility protocol."}},
{"grpc_port", {"GRPCThreads", "Number of threads in the server of the GRPC protocol."}}, {"grpc_port", {"GRPCThreads", "Number of threads in the server of the GRPC protocol."}},
{"prometheus.port", {"PrometheusThreads", "Number of threads in the server of the Prometheus endpoint. Note: prometheus endpoints can be also used via the usual HTTP/HTTPs ports."}} {"prometheus.port", {"PrometheusThreads", "Number of threads in the server of the Prometheus endpoint. Note: prometheus endpoints can be also used via the usual HTTP/HTTPs ports."}},
{"keeper_server.tcp_port", {"KeeperTCPThreads", "Number of threads in the server of the Keeper TCP protocol (without TLS)."}},
{"keeper_server.tcp_port_secure", {"KeeperTCPSecureThreads", "Number of threads in the server of the Keeper TCP protocol (with TLS)."}}
}; };
auto it = metric_map.find(name); auto it = metric_map.find(name);
if (it == metric_map.end()) if (it == metric_map.end())
@ -1707,102 +1434,14 @@ void AsynchronousMetrics::update(TimePoint update_time)
new_values[name_doc.first] = { server_metric.current_threads, name_doc.second }; new_values[name_doc.first] = { server_metric.current_threads, name_doc.second };
} }
} }
#if USE_NURAFT
{
auto keeper_dispatcher = getContext()->tryGetKeeperDispatcher();
if (keeper_dispatcher)
{
size_t is_leader = 0;
size_t is_follower = 0;
size_t is_observer = 0;
size_t is_standalone = 0;
size_t znode_count = 0;
size_t watch_count = 0;
size_t ephemerals_count = 0;
size_t approximate_data_size = 0;
size_t key_arena_size = 0;
size_t latest_snapshot_size = 0;
size_t open_file_descriptor_count = 0;
size_t max_file_descriptor_count = 0;
size_t followers = 0;
size_t synced_followers = 0;
size_t zxid = 0;
size_t session_with_watches = 0;
size_t paths_watched = 0;
size_t snapshot_dir_size = 0;
size_t log_dir_size = 0;
if (keeper_dispatcher->isServerActive())
{
auto keeper_info = keeper_dispatcher -> getKeeper4LWInfo();
is_standalone = static_cast<size_t>(keeper_info.is_standalone);
is_leader = static_cast<size_t>(keeper_info.is_leader);
is_observer = static_cast<size_t>(keeper_info.is_observer);
is_follower = static_cast<size_t>(keeper_info.is_follower);
zxid = keeper_info.last_zxid;
const auto & state_machine = keeper_dispatcher->getStateMachine();
znode_count = state_machine.getNodesCount();
watch_count = state_machine.getTotalWatchesCount();
ephemerals_count = state_machine.getTotalEphemeralNodesCount();
approximate_data_size = state_machine.getApproximateDataSize();
key_arena_size = state_machine.getKeyArenaSize();
latest_snapshot_size = state_machine.getLatestSnapshotBufSize();
session_with_watches = state_machine.getSessionsWithWatchesCount();
paths_watched = state_machine.getWatchedPathsCount();
snapshot_dir_size = keeper_dispatcher->getSnapDirSize();
log_dir_size = keeper_dispatcher->getLogDirSize();
#if defined(__linux__) || defined(__APPLE__)
open_file_descriptor_count = getCurrentProcessFDCount();
max_file_descriptor_count = getMaxFileDescriptorCount();
#endif
if (keeper_info.is_leader)
{
followers = keeper_info.follower_count;
synced_followers = keeper_info.synced_follower_count;
}
}
new_values["KeeperIsLeader"] = { is_leader, "1 if ClickHouse Keeper is a leader, 0 otherwise." };
new_values["KeeperIsFollower"] = { is_follower, "1 if ClickHouse Keeper is a follower, 0 otherwise." };
new_values["KeeperIsObserver"] = { is_observer, "1 if ClickHouse Keeper is an observer, 0 otherwise." };
new_values["KeeperIsStandalone"] = { is_standalone, "1 if ClickHouse Keeper is in a standalone mode, 0 otherwise." };
new_values["KeeperZnodeCount"] = { znode_count, "The number of nodes (data entries) in ClickHouse Keeper." };
new_values["KeeperWatchCount"] = { watch_count, "The number of watches in ClickHouse Keeper." };
new_values["KeeperEphemeralsCount"] = { ephemerals_count, "The number of ephemeral nodes in ClickHouse Keeper." };
new_values["KeeperApproximateDataSize"] = { approximate_data_size, "The approximate data size of ClickHouse Keeper, in bytes." };
new_values["KeeperKeyArenaSize"] = { key_arena_size, "The size in bytes of the memory arena for keys in ClickHouse Keeper." };
new_values["KeeperLatestSnapshotSize"] = { latest_snapshot_size, "The uncompressed size in bytes of the latest snapshot created by ClickHouse Keeper." };
new_values["KeeperOpenFileDescriptorCount"] = { open_file_descriptor_count, "The number of open file descriptors in ClickHouse Keeper." };
new_values["KeeperMaxFileDescriptorCount"] = { max_file_descriptor_count, "The maximum number of open file descriptors in ClickHouse Keeper." };
new_values["KeeperFollowers"] = { followers, "The number of followers of ClickHouse Keeper." };
new_values["KeeperSyncedFollowers"] = { synced_followers, "The number of followers of ClickHouse Keeper who are also in-sync." };
new_values["KeeperZxid"] = { zxid, "The current transaction id number (zxid) in ClickHouse Keeper." };
new_values["KeeperSessionWithWatches"] = { session_with_watches, "The number of client sessions of ClickHouse Keeper having watches." };
new_values["KeeperPathsWatched"] = { paths_watched, "The number of different paths watched by the clients of ClickHouse Keeper." };
new_values["KeeperSnapshotDirSize"] = { snapshot_dir_size, "The size of the snapshots directory of ClickHouse Keeper, in bytes." };
new_values["KeeperLogDirSize"] = { log_dir_size, "The size of the logs directory of ClickHouse Keeper, in bytes." };
}
}
#endif
updateHeavyMetricsIfNeeded(current_time, update_time, new_values);
/// Add more metrics as you wish. /// Add more metrics as you wish.
updateImpl(new_values, update_time, current_time);
new_values["AsynchronousMetricsCalculationTimeSpent"] = { watch.elapsedSeconds(), "Time in seconds spent for calculation of asynchronous metrics (this is the overhead of asynchronous metrics)." }; new_values["AsynchronousMetricsCalculationTimeSpent"] = { watch.elapsedSeconds(), "Time in seconds spent for calculation of asynchronous metrics (this is the overhead of asynchronous metrics)." };
/// Log the new metrics. logImpl(new_values);
if (auto asynchronous_metric_log = getContext()->getAsynchronousMetricLog())
{
asynchronous_metric_log->addValues(new_values);
}
first_run = false; first_run = false;
@ -1811,75 +1450,4 @@ void AsynchronousMetrics::update(TimePoint update_time)
values = new_values; values = new_values;
} }
void AsynchronousMetrics::updateDetachedPartsStats()
{
DetachedPartsStats current_values{};
for (const auto & db : DatabaseCatalog::instance().getDatabases())
{
if (!db.second->canContainMergeTreeTables())
continue;
for (auto iterator = db.second->getTablesIterator(getContext()); iterator->isValid(); iterator->next())
{
const auto & table = iterator->table();
if (!table)
continue;
if (MergeTreeData * table_merge_tree = dynamic_cast<MergeTreeData *>(table.get()))
{
for (const auto & detached_part: table_merge_tree->getDetachedParts())
{
if (!detached_part.valid_name)
continue;
if (detached_part.prefix.empty())
++current_values.detached_by_user;
++current_values.count;
}
}
}
}
detached_parts_stats = current_values;
}
void AsynchronousMetrics::updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, AsynchronousMetricValues & new_values)
{
const auto time_after_previous_update = current_time - heavy_metric_previous_update_time;
const bool update_heavy_metric = time_after_previous_update >= heavy_metric_update_period || first_run;
if (update_heavy_metric)
{
heavy_metric_previous_update_time = update_time;
Stopwatch watch;
/// Test shows that listing 100000 entries consuming around 0.15 sec.
updateDetachedPartsStats();
watch.stop();
/// Normally heavy metrics don't delay the rest of the metrics calculation
/// otherwise log the warning message
auto log_level = std::make_pair(DB::LogsLevel::trace, Poco::Message::PRIO_TRACE);
if (watch.elapsedSeconds() > (update_period.count() / 2.))
log_level = std::make_pair(DB::LogsLevel::debug, Poco::Message::PRIO_DEBUG);
else if (watch.elapsedSeconds() > (update_period.count() / 4. * 3))
log_level = std::make_pair(DB::LogsLevel::warning, Poco::Message::PRIO_WARNING);
LOG_IMPL(log, log_level.first, log_level.second,
"Update heavy metrics. "
"Update period {} sec. "
"Update heavy metrics period {} sec. "
"Heavy metrics calculation elapsed: {} sec.",
update_period.count(),
heavy_metric_update_period.count(),
watch.elapsedSeconds());
}
new_values["NumberOfDetachedParts"] = { detached_parts_stats.count, "The total number of parts detached from MergeTree tables. A part can be detached by a user with the `ALTER TABLE DETACH` query or by the server itself it the part is broken, unexpected or unneeded. The server does not care about detached parts and they can be removed." };
new_values["NumberOfDetachedByUserParts"] = { detached_parts_stats.detached_by_user, "The total number of parts detached from MergeTree tables by users with the `ALTER TABLE DETACH` query (as opposed to unexpected, broken or ignored parts). The server does not care about detached parts and they can be removed." };
}
} }

View File

@ -1,6 +1,5 @@
#pragma once #pragma once
#include <Interpreters/Context_fwd.h>
#include <Common/MemoryStatisticsOS.h> #include <Common/MemoryStatisticsOS.h>
#include <Common/ThreadPool.h> #include <Common/ThreadPool.h>
#include <Common/Stopwatch.h> #include <Common/Stopwatch.h>
@ -55,17 +54,15 @@ struct ProtocolServerMetrics
* All the values are either gauge type (like the total number of tables, the current memory usage). * All the values are either gauge type (like the total number of tables, the current memory usage).
* Or delta-counters representing some accumulation during the interval of time. * Or delta-counters representing some accumulation during the interval of time.
*/ */
class AsynchronousMetrics : WithContext class AsynchronousMetrics
{ {
public: public:
using ProtocolServerMetricsFunc = std::function<std::vector<ProtocolServerMetrics>()>; using ProtocolServerMetricsFunc = std::function<std::vector<ProtocolServerMetrics>()>;
AsynchronousMetrics( AsynchronousMetrics(
ContextPtr global_context_,
int update_period_seconds, int update_period_seconds,
int heavy_metrics_update_period_seconds,
const ProtocolServerMetricsFunc & protocol_server_metrics_func_); const ProtocolServerMetricsFunc & protocol_server_metrics_func_);
~AsynchronousMetrics(); virtual ~AsynchronousMetrics();
/// Separate method allows to initialize the `servers` variable beforehand. /// Separate method allows to initialize the `servers` variable beforehand.
void start(); void start();
@ -75,12 +72,22 @@ public:
/// Returns copy of all values. /// Returns copy of all values.
AsynchronousMetricValues getValues() const; AsynchronousMetricValues getValues() const;
private: protected:
using Duration = std::chrono::seconds; using Duration = std::chrono::seconds;
using TimePoint = std::chrono::system_clock::time_point; using TimePoint = std::chrono::system_clock::time_point;
const Duration update_period; const Duration update_period;
const Duration heavy_metric_update_period;
/// Some values are incremental and we have to calculate the difference.
/// On first run we will only collect the values to subtract later.
bool first_run = true;
TimePoint previous_update_time;
Poco::Logger * log;
private:
virtual void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) = 0;
virtual void logImpl(AsynchronousMetricValues &) {}
ProtocolServerMetricsFunc protocol_server_metrics_func; ProtocolServerMetricsFunc protocol_server_metrics_func;
mutable std::mutex mutex; mutable std::mutex mutex;
@ -88,20 +95,6 @@ private:
bool quit {false}; bool quit {false};
AsynchronousMetricValues values; AsynchronousMetricValues values;
/// Some values are incremental and we have to calculate the difference.
/// On first run we will only collect the values to subtract later.
bool first_run = true;
TimePoint previous_update_time;
TimePoint heavy_metric_previous_update_time;
struct DetachedPartsStats
{
size_t count;
size_t detached_by_user;
};
DetachedPartsStats detached_parts_stats{};
#if defined(OS_LINUX) || defined(OS_FREEBSD) #if defined(OS_LINUX) || defined(OS_FREEBSD)
MemoryStatisticsOS memory_stat; MemoryStatisticsOS memory_stat;
#endif #endif
@ -212,11 +205,6 @@ private:
void run(); void run();
void update(TimePoint update_time); void update(TimePoint update_time);
void updateDetachedPartsStats();
void updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, AsynchronousMetricValues & new_values);
Poco::Logger * log;
}; };
} }

View File

@ -2,7 +2,7 @@
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <Common/VariableContext.h> #include <Common/VariableContext.h>
#include <Interpreters/TraceCollector.h> #include <Common/TraceSender.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/LockMemoryExceptionInThread.h> #include <Common/LockMemoryExceptionInThread.h>
#include <Common/MemoryTrackerBlockerInThread.h> #include <Common/MemoryTrackerBlockerInThread.h>
@ -178,7 +178,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT
if (unlikely(current_profiler_limit && will_be > current_profiler_limit)) if (unlikely(current_profiler_limit && will_be > current_profiler_limit))
{ {
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
DB::TraceCollector::collect(DB::TraceType::Memory, StackTrace(), size); DB::TraceSender::send(DB::TraceType::Memory, StackTrace(), {.size = size});
setOrRaiseProfilerLimit((will_be + profiler_step - 1) / profiler_step * profiler_step); setOrRaiseProfilerLimit((will_be + profiler_step - 1) / profiler_step * profiler_step);
allocation_traced = true; allocation_traced = true;
} }
@ -187,7 +187,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT
if (unlikely(sample_probability > 0.0 && sample(thread_local_rng))) if (unlikely(sample_probability > 0.0 && sample(thread_local_rng)))
{ {
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
DB::TraceCollector::collect(DB::TraceType::MemorySample, StackTrace(), size); DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = size});
allocation_traced = true; allocation_traced = true;
} }
@ -305,7 +305,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT
if (peak_updated && allocation_traced) if (peak_updated && allocation_traced)
{ {
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
DB::TraceCollector::collect(DB::TraceType::MemoryPeak, StackTrace(), will_be); DB::TraceSender::send(DB::TraceType::MemoryPeak, StackTrace(), {.size = will_be});
} }
if (auto * loaded_next = parent.load(std::memory_order_relaxed)) if (auto * loaded_next = parent.load(std::memory_order_relaxed))
@ -361,7 +361,7 @@ void MemoryTracker::free(Int64 size)
if (unlikely(sample_probability > 0.0 && sample(thread_local_rng))) if (unlikely(sample_probability > 0.0 && sample(thread_local_rng)))
{ {
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
DB::TraceCollector::collect(DB::TraceType::MemorySample, StackTrace(), -size); DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = -size});
} }
Int64 accounted_size = size; Int64 accounted_size = size;

View File

@ -1,5 +1,6 @@
#include <Common/ProfileEvents.h> #include <Common/ProfileEvents.h>
#include <Common/CurrentThread.h> #include <Common/CurrentThread.h>
#include <Common/TraceSender.h>
/// Available events. Add something here as you wish. /// Available events. Add something here as you wish.
@ -433,6 +434,15 @@ The server successfully detected this situation and will download merged part fr
M(KeeperSnapshotApplysFailed, "Number of failed snapshot applying")\ M(KeeperSnapshotApplysFailed, "Number of failed snapshot applying")\
M(KeeperReadSnapshot, "Number of snapshot read(serialization)")\ M(KeeperReadSnapshot, "Number of snapshot read(serialization)")\
M(KeeperSaveSnapshot, "Number of snapshot save")\ M(KeeperSaveSnapshot, "Number of snapshot save")\
M(KeeperCreateRequest, "Number of create requests")\
M(KeeperRemoveRequest, "Number of remove requests")\
M(KeeperSetRequest, "Number of set requests")\
M(KeeperCheckRequest, "Number of check requests")\
M(KeeperMultiRequest, "Number of multi requests")\
M(KeeperMultiReadRequest, "Number of multi read requests")\
M(KeeperGetRequest, "Number of get requests")\
M(KeeperListRequest, "Number of list requests")\
M(KeeperExistsRequest, "Number of exists requests")\
\ \
M(OverflowBreak, "Number of times, data processing was cancelled by query complexity limitation with setting '*_overflow_mode' = 'break' and the result is incomplete.") \ M(OverflowBreak, "Number of times, data processing was cancelled by query complexity limitation with setting '*_overflow_mode' = 'break' and the result is incomplete.") \
M(OverflowThrow, "Number of times, data processing was cancelled by query complexity limitation with setting '*_overflow_mode' = 'throw' and exception was thrown.") \ M(OverflowThrow, "Number of times, data processing was cancelled by query complexity limitation with setting '*_overflow_mode' = 'throw' and exception was thrown.") \
@ -514,15 +524,29 @@ const char * getDocumentation(Event event)
return strings[event]; return strings[event];
} }
Event end() { return END; } Event end() { return END; }
void increment(Event event, Count amount) void increment(Event event, Count amount)
{ {
DB::CurrentThread::getProfileEvents().increment(event, amount); DB::CurrentThread::getProfileEvents().increment(event, amount);
} }
void Counters::increment(Event event, Count amount)
{
Counters * current = this;
bool send_to_trace_log = false;
do
{
send_to_trace_log |= current->trace_profile_events;
current->counters[event].fetch_add(amount, std::memory_order_relaxed);
current = current->parent;
} while (current != nullptr);
if (unlikely(send_to_trace_log))
DB::TraceSender::send(DB::TraceType::ProfileEvent, StackTrace(), {.event = event, .increment = amount});
}
CountersIncrement::CountersIncrement(Counters::Snapshot const & snapshot) CountersIncrement::CountersIncrement(Counters::Snapshot const & snapshot)
{ {
init(); init();

View File

@ -25,10 +25,12 @@ namespace ProfileEvents
class Counters class Counters
{ {
private:
Counter * counters = nullptr; Counter * counters = nullptr;
std::unique_ptr<Counter[]> counters_holder; std::unique_ptr<Counter[]> counters_holder;
/// Used to propagate increments /// Used to propagate increments
Counters * parent = nullptr; Counters * parent = nullptr;
bool trace_profile_events = false;
public: public:
@ -51,15 +53,7 @@ namespace ProfileEvents
return counters[event]; return counters[event];
} }
inline void increment(Event event, Count amount = 1) void increment(Event event, Count amount = 1);
{
Counters * current = this;
do
{
current->counters[event].fetch_add(amount, std::memory_order_relaxed);
current = current->parent;
} while (current != nullptr);
}
struct Snapshot struct Snapshot
{ {
@ -97,6 +91,11 @@ namespace ProfileEvents
parent = parent_; parent = parent_;
} }
void setTraceProfileEvents(bool value)
{
trace_profile_events = value;
}
/// Set all counters to zero /// Set all counters to zero
void resetCounters(); void resetCounters();

View File

@ -1,7 +1,7 @@
#include "QueryProfiler.h" #include "QueryProfiler.h"
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <Interpreters/TraceCollector.h> #include <Common/TraceSender.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/StackTrace.h> #include <Common/StackTrace.h>
#include <Common/thread_local_rng.h> #include <Common/thread_local_rng.h>
@ -66,7 +66,7 @@ namespace
const auto signal_context = *reinterpret_cast<ucontext_t *>(context); const auto signal_context = *reinterpret_cast<ucontext_t *>(context);
const StackTrace stack_trace(signal_context); const StackTrace stack_trace(signal_context);
TraceCollector::collect(trace_type, stack_trace, 0); TraceSender::send(trace_type, stack_trace, {});
ProfileEvents::increment(ProfileEvents::QueryProfilerRuns); ProfileEvents::increment(ProfileEvents::QueryProfilerRuns);
errno = saved_errno; errno = saved_errno;

View File

@ -14,7 +14,7 @@ namespace
/// The performance test query ids can be surprisingly long like /// The performance test query ids can be surprisingly long like
/// `aggregating_merge_tree_simple_aggregate_function_string.query100.profile100`, /// `aggregating_merge_tree_simple_aggregate_function_string.query100.profile100`,
/// so make some allowance for them as well. /// so make some allowance for them as well.
constexpr size_t QUERY_ID_MAX_LEN = 128; constexpr size_t QUERY_ID_MAX_LEN = 100;
static_assert(QUERY_ID_MAX_LEN <= std::numeric_limits<uint8_t>::max()); static_assert(QUERY_ID_MAX_LEN <= std::numeric_limits<uint8_t>::max());
} }
@ -23,7 +23,7 @@ namespace DB
LazyPipeFDs TraceSender::pipe; LazyPipeFDs TraceSender::pipe;
void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Int64 size) void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Extras extras)
{ {
constexpr size_t buf_size = sizeof(char) /// TraceCollector stop flag constexpr size_t buf_size = sizeof(char) /// TraceCollector stop flag
+ sizeof(UInt8) /// String size + sizeof(UInt8) /// String size
@ -32,12 +32,14 @@ void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Int
+ sizeof(StackTrace::FramePointers) /// Collected stack trace, maximum capacity + sizeof(StackTrace::FramePointers) /// Collected stack trace, maximum capacity
+ sizeof(TraceType) /// trace type + sizeof(TraceType) /// trace type
+ sizeof(UInt64) /// thread_id + sizeof(UInt64) /// thread_id
+ sizeof(Int64); /// size + sizeof(Int64) /// size
+ sizeof(ProfileEvents::Event) /// event
+ sizeof(ProfileEvents::Count); /// increment
/// Write should be atomic to avoid overlaps /// Write should be atomic to avoid overlaps
/// (since recursive collect() is possible) /// (since recursive collect() is possible)
static_assert(PIPE_BUF >= 512); static_assert(PIPE_BUF >= 512);
static_assert(buf_size <= 512, "Only write of PIPE_BUF to pipe is atomic and the minimal known PIPE_BUF across supported platforms is 512"); static_assert(buf_size <= PIPE_BUF, "Only write of PIPE_BUF to pipe is atomic and the minimal known PIPE_BUF across supported platforms is 512");
char buffer[buf_size]; char buffer[buf_size];
WriteBufferFromFileDescriptorDiscardOnFailure out(pipe.fds_rw[1], buf_size, buffer); WriteBufferFromFileDescriptorDiscardOnFailure out(pipe.fds_rw[1], buf_size, buffer);
@ -71,7 +73,9 @@ void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Int
writePODBinary(trace_type, out); writePODBinary(trace_type, out);
writePODBinary(thread_id, out); writePODBinary(thread_id, out);
writePODBinary(size, out); writePODBinary(extras.size, out);
writePODBinary(extras.event, out);
writePODBinary(extras.increment, out);
out.next(); out.next();
} }

View File

@ -1,6 +1,7 @@
#pragma once #pragma once
#include <Common/PipeFDs.h> #include <Common/PipeFDs.h>
#include <Common/ProfileEvents.h>
#include <base/types.h> #include <base/types.h>
class StackTrace; class StackTrace;
@ -17,6 +18,7 @@ enum class TraceType : uint8_t
Memory, Memory,
MemorySample, MemorySample,
MemoryPeak, MemoryPeak,
ProfileEvent,
}; };
/// This is the second part of TraceCollector, that sends stacktrace to the pipe. /// This is the second part of TraceCollector, that sends stacktrace to the pipe.
@ -24,10 +26,18 @@ enum class TraceType : uint8_t
class TraceSender class TraceSender
{ {
public: public:
struct Extras
{
/// size - for memory tracing is the amount of memory allocated; for other trace types it is 0.
Int64 size{};
/// Event type and increment for 'ProfileEvent' trace type; for other trace types defaults.
ProfileEvents::Event event{ProfileEvents::end()};
ProfileEvents::Count increment{};
};
/// Collect a stack trace. This method is signal safe. /// Collect a stack trace. This method is signal safe.
/// Precondition: the TraceCollector object must be created. /// Precondition: the TraceCollector object must be created.
/// size - for memory tracing is the amount of memory allocated; for other trace types it is 0. static void send(TraceType trace_type, const StackTrace & stack_trace, Extras extras);
static void send(TraceType trace_type, const StackTrace & stack_trace, Int64 size);
private: private:
friend class TraceCollector; friend class TraceCollector;

View File

@ -42,15 +42,15 @@ public:
return s; return s;
} }
template <typename ValueType, typename ParseFunction> template <typename ValueType, bool ReturnDefault, typename ParseFunction>
static ValueType getValue(const Node * node, const std::string & path, static ValueType getValue(const Node * node, const std::string & path,
const std::optional<ValueType> & default_value, const ParseFunction & parse_function) const ValueType & default_value, const ParseFunction & parse_function)
{ {
const auto * value_node = node->getNodeByPath(path); const auto * value_node = node->getNodeByPath(path);
if (!value_node) if (!value_node)
{ {
if (default_value) if constexpr (ReturnDefault)
return *default_value; return default_value;
else else
throw Poco::NotFoundException(path); throw Poco::NotFoundException(path);
} }
@ -59,34 +59,64 @@ public:
}; };
std::string getString(const Node * node, const std::string & path, const std::optional<std::string> & default_value) std::string getString(const Node * node, const std::string & path)
{ {
return ParseHelper::getValue<std::string>(node, path, default_value, ParseHelper::parseString); return ParseHelper::getValue<std::string, false>(node, path, {}, ParseHelper::parseString);
} }
Int64 getInt64(const Node * node, const std::string & path, const std::optional<Int64> & default_value) std::string getString(const Node * node, const std::string & path, const std::string & default_value)
{ {
return ParseHelper::getValue<Int64>(node, path, default_value, ParseHelper::parseInt64); return ParseHelper::getValue<std::string, true>(node, path, default_value, ParseHelper::parseString);
} }
UInt64 getUInt64(const Node * node, const std::string & path, const std::optional<UInt64> & default_value) Int64 getInt64(const Node * node, const std::string & path)
{ {
return ParseHelper::getValue<UInt64>(node, path, default_value, ParseHelper::parseUInt64); return ParseHelper::getValue<Int64, false>(node, path, {}, ParseHelper::parseInt64);
} }
int getInt(const Node * node, const std::string & path, const std::optional<int> & default_value) Int64 getInt64(const Node * node, const std::string & path, Int64 default_value)
{ {
return ParseHelper::getValue<int>(node, path, default_value, ParseHelper::parseInt); return ParseHelper::getValue<Int64, true>(node, path, default_value, ParseHelper::parseInt64);
} }
unsigned getUInt(const Node * node, const std::string & path, const std::optional<unsigned> & default_value) UInt64 getUInt64(const Node * node, const std::string & path)
{ {
return ParseHelper::getValue<unsigned>(node, path, default_value, ParseHelper::parseUInt); return ParseHelper::getValue<UInt64, false>(node, path, {}, ParseHelper::parseUInt64);
} }
bool getBool(const Node * node, const std::string & path, const std::optional<bool> & default_value) UInt64 getUInt64(const Node * node, const std::string & path, UInt64 default_value)
{ {
return ParseHelper::getValue<bool>(node, path, default_value, ParseHelper::parseBool); return ParseHelper::getValue<UInt64, true>(node, path, default_value, ParseHelper::parseUInt64);
}
int getInt(const Node * node, const std::string & path)
{
return ParseHelper::getValue<int, false>(node, path, {}, ParseHelper::parseInt);
}
int getInt(const Node * node, const std::string & path, int default_value)
{
return ParseHelper::getValue<int, true>(node, path, default_value, ParseHelper::parseInt);
}
unsigned getUInt(const Node * node, const std::string & path)
{
return ParseHelper::getValue<unsigned, false>(node, path, {}, ParseHelper::parseUInt);
}
unsigned getUInt(const Node * node, const std::string & path, unsigned default_value)
{
return ParseHelper::getValue<unsigned, true>(node, path, default_value, ParseHelper::parseUInt);
}
bool getBool(const Node * node, const std::string & path)
{
return ParseHelper::getValue<bool, false>(node, path, {}, ParseHelper::parseBool);
}
bool getBool(const Node * node, const std::string & path, bool default_value)
{
return ParseHelper::getValue<bool, true>(node, path, default_value, ParseHelper::parseBool);
} }
} }

View File

@ -7,17 +7,26 @@
namespace DB:: XMLUtils namespace DB:: XMLUtils
{ {
/// Returns root element of the document.
Poco::XML::Node * getRootNode(Poco::XML::Document * document); Poco::XML::Node * getRootNode(Poco::XML::Document * document);
std::string getString(const Poco::XML::Node * node, const std::string & path, const std::optional<std::string> & default_value = std::nullopt); /// Finds the element in the node's subtree by the specified path and returns its inner text
/// trying to parse it as the requested type.
/// Throws an exception if path is not found.
std::string getString(const Poco::XML::Node * node, const std::string & path);
Int64 getInt64(const Poco::XML::Node * node, const std::string & path);
UInt64 getUInt64(const Poco::XML::Node * node, const std::string & path);
int getInt(const Poco::XML::Node * node, const std::string & path);
unsigned getUInt(const Poco::XML::Node * node, const std::string & path);
bool getBool(const Poco::XML::Node * node, const std::string & path);
Int64 getInt64(const Poco::XML::Node * node, const std::string & path, const std::optional<Int64> & default_value = std::nullopt); /// Finds the element in the node's subtree by the specified path and returns its inner text
/// trying to parse it as the requested type.
UInt64 getUInt64(const Poco::XML::Node * node, const std::string & path, const std::optional<UInt64> & default_value = std::nullopt); /// Returns the specified default value if path is not found.
std::string getString(const Poco::XML::Node * node, const std::string & path, const std::string & default_value);
int getInt(const Poco::XML::Node * node, const std::string & path, const std::optional<int> & default_value = std::nullopt); Int64 getInt64(const Poco::XML::Node * node, const std::string & path, Int64 default_value);
UInt64 getUInt64(const Poco::XML::Node * node, const std::string & path, UInt64 default_value);
unsigned getUInt(const Poco::XML::Node * node, const std::string & path, const std::optional<unsigned> & default_value = std::nullopt); int getInt(const Poco::XML::Node * node, const std::string & path, int default_value);
unsigned getUInt(const Poco::XML::Node * node, const std::string & path, unsigned default_value);
bool getBool(const Poco::XML::Node * node, const std::string & path, const std::optional<bool> & default_value = std::nullopt); bool getBool(const Poco::XML::Node * node, const std::string & path, bool default_value);
} }

View File

@ -466,7 +466,7 @@ void ZooKeeper::connect(
} }
else else
{ {
LOG_TEST(log, "Connected to ZooKeeper at {} with session_id {}{}", socket.peerAddress().toString(), session_id, fail_reasons.str()); LOG_INFO(log, "Connected to ZooKeeper at {} with session_id {}{}", socket.peerAddress().toString(), session_id, fail_reasons.str());
} }
} }
@ -867,12 +867,12 @@ void ZooKeeper::finalize(bool error_send, bool error_receive, const String & rea
/// If some thread (send/receive) already finalizing session don't try to do it /// If some thread (send/receive) already finalizing session don't try to do it
bool already_started = finalization_started.test_and_set(); bool already_started = finalization_started.test_and_set();
LOG_TEST(log, "Finalizing session {}: finalization_started={}, queue_finished={}, reason={}",
session_id, already_started, requests_queue.isFinished(), reason);
if (already_started) if (already_started)
return; return;
LOG_INFO(log, "Finalizing session {}: finalization_started={}, queue_finished={}, reason={}",
session_id, already_started, requests_queue.isFinished(), reason);
auto expire_session_if_not_expired = [&] auto expire_session_if_not_expired = [&]
{ {
/// No new requests will appear in queue after finish() /// No new requests will appear in queue after finish()

View File

@ -280,6 +280,7 @@ Changelog::Changelog(
, force_sync(force_sync_) , force_sync(force_sync_)
, log(log_) , log(log_)
, compress_logs(compress_logs_) , compress_logs(compress_logs_)
, write_operations(std::numeric_limits<size_t>::max())
{ {
/// Load all files in changelog directory /// Load all files in changelog directory
namespace fs = std::filesystem; namespace fs = std::filesystem;
@ -299,10 +300,13 @@ Changelog::Changelog(
LOG_WARNING(log, "No logs exists in {}. It's Ok if it's the first run of clickhouse-keeper.", changelogs_dir.generic_string()); LOG_WARNING(log, "No logs exists in {}. It's Ok if it's the first run of clickhouse-keeper.", changelogs_dir.generic_string());
clean_log_thread = ThreadFromGlobalPool([this] { cleanLogThread(); }); clean_log_thread = ThreadFromGlobalPool([this] { cleanLogThread(); });
write_thread = ThreadFromGlobalPool([this] { writeThread(); });
} }
void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uint64_t logs_to_keep) void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uint64_t logs_to_keep)
{ {
std::lock_guard writer_lock(writer_mutex);
std::optional<ChangelogReadResult> last_log_read_result; std::optional<ChangelogReadResult> last_log_read_result;
/// Last log has some free space to write /// Last log has some free space to write
@ -336,7 +340,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin
removeAllLogs(); removeAllLogs();
min_log_id = last_commited_log_index; min_log_id = last_commited_log_index;
max_log_id = last_commited_log_index == 0 ? 0 : last_commited_log_index - 1; max_log_id = last_commited_log_index == 0 ? 0 : last_commited_log_index - 1;
rotate(max_log_id + 1); rotate(max_log_id + 1, writer_lock);
return; return;
} }
else if (changelog_description.from_log_index > start_to_read_from) else if (changelog_description.from_log_index > start_to_read_from)
@ -427,7 +431,9 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin
/// Start new log if we don't initialize writer from previous log. All logs can be "complete". /// Start new log if we don't initialize writer from previous log. All logs can be "complete".
if (!current_writer) if (!current_writer)
rotate(max_log_id + 1); rotate(max_log_id + 1, writer_lock);
initialized = true;
} }
@ -500,10 +506,11 @@ void Changelog::removeAllLogs()
logs.clear(); logs.clear();
} }
void Changelog::rotate(uint64_t new_start_log_index) void Changelog::rotate(uint64_t new_start_log_index, std::lock_guard<std::mutex> &)
{ {
/// Flush previous log /// Flush previous log
flush(); if (current_writer)
current_writer->flush(force_sync);
/// Start new one /// Start new one
ChangelogFileDescription new_description; ChangelogFileDescription new_description;
@ -540,27 +547,72 @@ ChangelogRecord Changelog::buildRecord(uint64_t index, const LogEntryPtr & log_e
return record; return record;
} }
void Changelog::writeThread()
{
WriteOperation write_operation;
while (write_operations.pop(write_operation))
{
assert(initialized);
if (auto * append_log = std::get_if<AppendLog>(&write_operation))
{
std::lock_guard writer_lock(writer_mutex);
assert(current_writer);
const auto & current_changelog_description = existing_changelogs[current_writer->getStartIndex()];
const bool log_is_complete = append_log->index - current_writer->getStartIndex() == current_changelog_description.expectedEntriesCountInLog();
if (log_is_complete)
rotate(append_log->index, writer_lock);
current_writer->appendRecord(buildRecord(append_log->index, append_log->log_entry));
}
else
{
const auto & flush = std::get<Flush>(write_operation);
{
std::lock_guard writer_lock(writer_mutex);
if (current_writer)
current_writer->flush(force_sync);
}
{
std::lock_guard lock{durable_idx_mutex};
last_durable_idx = flush.index;
}
durable_idx_cv.notify_all();
// we shouldn't start the raft_server before sending it here
if (auto raft_server_locked = raft_server.lock())
raft_server_locked->notify_log_append_completion(true);
else
LOG_WARNING(log, "Raft server is not set in LogStore.");
}
}
}
void Changelog::appendEntry(uint64_t index, const LogEntryPtr & log_entry) void Changelog::appendEntry(uint64_t index, const LogEntryPtr & log_entry)
{ {
if (!current_writer) if (!initialized)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Changelog must be initialized before appending records"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Changelog must be initialized before appending records");
if (logs.empty()) if (logs.empty())
min_log_id = index; min_log_id = index;
const auto & current_changelog_description = existing_changelogs[current_writer->getStartIndex()];
const bool log_is_complete = index - current_writer->getStartIndex() == current_changelog_description.expectedEntriesCountInLog();
if (log_is_complete)
rotate(index);
current_writer->appendRecord(buildRecord(index, log_entry));
logs[index] = log_entry; logs[index] = log_entry;
max_log_id = index; max_log_id = index;
if (!write_operations.tryPush(AppendLog{index, log_entry}))
LOG_WARNING(log, "Changelog is shut down");
} }
void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry) void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry)
{ {
{
std::lock_guard lock(writer_mutex);
/// This write_at require to overwrite everything in this file and also in previous file(s) /// This write_at require to overwrite everything in this file and also in previous file(s)
const bool go_to_previous_file = index < current_writer->getStartIndex(); const bool go_to_previous_file = index < current_writer->getStartIndex();
@ -586,6 +638,7 @@ void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry)
itr = existing_changelogs.erase(itr); itr = existing_changelogs.erase(itr);
} }
} }
}
/// Remove redundant logs from memory /// Remove redundant logs from memory
/// Everything >= index must be removed /// Everything >= index must be removed
@ -597,6 +650,7 @@ void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry)
void Changelog::compact(uint64_t up_to_log_index) void Changelog::compact(uint64_t up_to_log_index)
{ {
std::lock_guard lock(writer_mutex);
LOG_INFO(log, "Compact logs up to log index {}, our max log id is {}", up_to_log_index, max_log_id); LOG_INFO(log, "Compact logs up to log index {}, our max log id is {}", up_to_log_index, max_log_id);
bool remove_all_logs = false; bool remove_all_logs = false;
@ -643,7 +697,7 @@ void Changelog::compact(uint64_t up_to_log_index)
std::erase_if(logs, [up_to_log_index] (const auto & item) { return item.first <= up_to_log_index; }); std::erase_if(logs, [up_to_log_index] (const auto & item) { return item.first <= up_to_log_index; });
if (need_rotate) if (need_rotate)
rotate(up_to_log_index + 1); rotate(up_to_log_index + 1, lock);
LOG_INFO(log, "Compaction up to {} finished new min index {}, new max index {}", up_to_log_index, min_log_id, max_log_id); LOG_INFO(log, "Compaction up to {} finished new min index {}, new max index {}", up_to_log_index, min_log_id, max_log_id);
} }
@ -747,8 +801,19 @@ void Changelog::applyEntriesFromBuffer(uint64_t index, nuraft::buffer & buffer)
void Changelog::flush() void Changelog::flush()
{ {
if (current_writer) if (flushAsync())
current_writer->flush(force_sync); {
std::unique_lock lock{durable_idx_mutex};
durable_idx_cv.wait(lock, [&] { return last_durable_idx == max_log_id; });
}
}
bool Changelog::flushAsync()
{
bool pushed = write_operations.push(Flush{max_log_id});
if (!pushed)
LOG_WARNING(log, "Changelog is shut down");
return pushed;
} }
void Changelog::shutdown() void Changelog::shutdown()
@ -758,6 +823,12 @@ void Changelog::shutdown()
if (clean_log_thread.joinable()) if (clean_log_thread.joinable())
clean_log_thread.join(); clean_log_thread.join();
if (!write_operations.isFinished())
write_operations.finish();
if (write_thread.joinable())
write_thread.join();
} }
Changelog::~Changelog() Changelog::~Changelog()
@ -789,4 +860,10 @@ void Changelog::cleanLogThread()
} }
} }
void Changelog::setRaftServer(const nuraft::ptr<nuraft::raft_server> & raft_server_)
{
assert(raft_server_);
raft_server = raft_server_;
}
} }

View File

@ -1,8 +1,10 @@
#pragma once #pragma once
#include <libnuraft/nuraft.hxx> #include <libnuraft/nuraft.hxx>
#include <libnuraft/raft_server.hxx>
#include <city.h> #include <city.h>
#include <optional> #include <optional>
#include <base/defines.h>
#include <IO/WriteBufferFromFile.h> #include <IO/WriteBufferFromFile.h>
#include <IO/HashingWriteBuffer.h> #include <IO/HashingWriteBuffer.h>
#include <IO/CompressionMethod.h> #include <IO/CompressionMethod.h>
@ -121,6 +123,8 @@ public:
/// Fsync latest log to disk and flush buffer /// Fsync latest log to disk and flush buffer
void flush(); void flush();
bool flushAsync();
void shutdown(); void shutdown();
uint64_t size() const uint64_t size() const
@ -128,6 +132,14 @@ public:
return logs.size(); return logs.size();
} }
uint64_t lastDurableIndex() const
{
std::lock_guard lock{durable_idx_mutex};
return last_durable_idx;
}
void setRaftServer(const nuraft::ptr<nuraft::raft_server> & raft_server_);
/// Fsync log to disk /// Fsync log to disk
~Changelog(); ~Changelog();
@ -136,7 +148,7 @@ private:
static ChangelogRecord buildRecord(uint64_t index, const LogEntryPtr & log_entry); static ChangelogRecord buildRecord(uint64_t index, const LogEntryPtr & log_entry);
/// Starts new file [new_start_log_index, new_start_log_index + rotate_interval] /// Starts new file [new_start_log_index, new_start_log_index + rotate_interval]
void rotate(uint64_t new_start_log_index); void rotate(uint64_t new_start_log_index, std::lock_guard<std::mutex> & writer_lock);
/// Currently existing changelogs /// Currently existing changelogs
std::map<uint64_t, ChangelogFileDescription> existing_changelogs; std::map<uint64_t, ChangelogFileDescription> existing_changelogs;
@ -162,7 +174,7 @@ private:
Poco::Logger * log; Poco::Logger * log;
bool compress_logs; bool compress_logs;
std::mutex writer_mutex;
/// Current writer for changelog file /// Current writer for changelog file
std::unique_ptr<ChangelogWriter> current_writer; std::unique_ptr<ChangelogWriter> current_writer;
/// Mapping log_id -> log_entry /// Mapping log_id -> log_entry
@ -175,6 +187,33 @@ private:
/// 128 is enough, even if log is not removed, it's not a problem /// 128 is enough, even if log is not removed, it's not a problem
ConcurrentBoundedQueue<std::string> log_files_to_delete_queue{128}; ConcurrentBoundedQueue<std::string> log_files_to_delete_queue{128};
ThreadFromGlobalPool clean_log_thread; ThreadFromGlobalPool clean_log_thread;
struct AppendLog
{
uint64_t index;
nuraft::ptr<nuraft::log_entry> log_entry;
};
struct Flush
{
uint64_t index;
};
using WriteOperation = std::variant<AppendLog, Flush>;
void writeThread();
ThreadFromGlobalPool write_thread;
ConcurrentBoundedQueue<WriteOperation> write_operations;
// last_durable_index needs to be exposed through const getter so we make mutex mutable
mutable std::mutex durable_idx_mutex;
std::condition_variable durable_idx_cv;
uint64_t last_durable_idx{0};
nuraft::wptr<nuraft::raft_server> raft_server;
bool initialized = false;
}; };
} }

View File

@ -0,0 +1,127 @@
#include <Coordination/KeeperAsynchronousMetrics.h>
#include <Coordination/KeeperDispatcher.h>
#include <Common/getCurrentProcessFDCount.h>
#include <Common/getMaxFileDescriptorCount.h>
namespace DB
{
void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousMetricValues & new_values)
{
#if USE_NURAFT
size_t is_leader = 0;
size_t is_follower = 0;
size_t is_observer = 0;
size_t is_standalone = 0;
size_t znode_count = 0;
size_t watch_count = 0;
size_t ephemerals_count = 0;
size_t approximate_data_size = 0;
size_t key_arena_size = 0;
size_t latest_snapshot_size = 0;
size_t open_file_descriptor_count = 0;
size_t max_file_descriptor_count = 0;
size_t followers = 0;
size_t synced_followers = 0;
size_t zxid = 0;
size_t session_with_watches = 0;
size_t paths_watched = 0;
size_t snapshot_dir_size = 0;
size_t log_dir_size = 0;
if (keeper_dispatcher.isServerActive())
{
auto keeper_info = keeper_dispatcher.getKeeper4LWInfo();
is_standalone = static_cast<size_t>(keeper_info.is_standalone);
is_leader = static_cast<size_t>(keeper_info.is_leader);
is_observer = static_cast<size_t>(keeper_info.is_observer);
is_follower = static_cast<size_t>(keeper_info.is_follower);
zxid = keeper_info.last_zxid;
const auto & state_machine = keeper_dispatcher.getStateMachine();
znode_count = state_machine.getNodesCount();
watch_count = state_machine.getTotalWatchesCount();
ephemerals_count = state_machine.getTotalEphemeralNodesCount();
approximate_data_size = state_machine.getApproximateDataSize();
key_arena_size = state_machine.getKeyArenaSize();
latest_snapshot_size = state_machine.getLatestSnapshotBufSize();
session_with_watches = state_machine.getSessionsWithWatchesCount();
paths_watched = state_machine.getWatchedPathsCount();
snapshot_dir_size = keeper_dispatcher.getSnapDirSize();
log_dir_size = keeper_dispatcher.getLogDirSize();
# if defined(__linux__) || defined(__APPLE__)
open_file_descriptor_count = getCurrentProcessFDCount();
max_file_descriptor_count = getMaxFileDescriptorCount();
# endif
if (keeper_info.is_leader)
{
followers = keeper_info.follower_count;
synced_followers = keeper_info.synced_follower_count;
}
}
new_values["KeeperIsLeader"] = { is_leader, "1 if ClickHouse Keeper is a leader, 0 otherwise." };
new_values["KeeperIsFollower"] = { is_follower, "1 if ClickHouse Keeper is a follower, 0 otherwise." };
new_values["KeeperIsObserver"] = { is_observer, "1 if ClickHouse Keeper is an observer, 0 otherwise." };
new_values["KeeperIsStandalone"] = { is_standalone, "1 if ClickHouse Keeper is in a standalone mode, 0 otherwise." };
new_values["KeeperZnodeCount"] = { znode_count, "The number of nodes (data entries) in ClickHouse Keeper." };
new_values["KeeperWatchCount"] = { watch_count, "The number of watches in ClickHouse Keeper." };
new_values["KeeperEphemeralsCount"] = { ephemerals_count, "The number of ephemeral nodes in ClickHouse Keeper." };
new_values["KeeperApproximateDataSize"] = { approximate_data_size, "The approximate data size of ClickHouse Keeper, in bytes." };
new_values["KeeperKeyArenaSize"] = { key_arena_size, "The size in bytes of the memory arena for keys in ClickHouse Keeper." };
new_values["KeeperLatestSnapshotSize"] = { latest_snapshot_size, "The uncompressed size in bytes of the latest snapshot created by ClickHouse Keeper." };
new_values["KeeperOpenFileDescriptorCount"] = { open_file_descriptor_count, "The number of open file descriptors in ClickHouse Keeper." };
new_values["KeeperMaxFileDescriptorCount"] = { max_file_descriptor_count, "The maximum number of open file descriptors in ClickHouse Keeper." };
new_values["KeeperFollowers"] = { followers, "The number of followers of ClickHouse Keeper." };
new_values["KeeperSyncedFollowers"] = { synced_followers, "The number of followers of ClickHouse Keeper who are also in-sync." };
new_values["KeeperZxid"] = { zxid, "The current transaction id number (zxid) in ClickHouse Keeper." };
new_values["KeeperSessionWithWatches"] = { session_with_watches, "The number of client sessions of ClickHouse Keeper having watches." };
new_values["KeeperPathsWatched"] = { paths_watched, "The number of different paths watched by the clients of ClickHouse Keeper." };
new_values["KeeperSnapshotDirSize"] = { snapshot_dir_size, "The size of the snapshots directory of ClickHouse Keeper, in bytes." };
new_values["KeeperLogDirSize"] = { log_dir_size, "The size of the logs directory of ClickHouse Keeper, in bytes." };
auto keeper_log_info = keeper_dispatcher.getKeeperLogInfo();
new_values["KeeperLastLogIdx"] = { keeper_log_info.last_log_idx, "Index of the last log stored in ClickHouse Keeper." };
new_values["KeeperLastLogTerm"] = { keeper_log_info.last_log_term, "Raft term of the last log stored in ClickHouse Keeper." };
new_values["KeeperLastCommittedLogIdx"] = { keeper_log_info.last_committed_log_idx, "Index of the last committed log in ClickHouse Keeper." };
new_values["KeeperTargetCommitLogIdx"] = { keeper_log_info.target_committed_log_idx, "Index until which logs can be committed in ClickHouse Keeper." };
new_values["KeeperLastSnapshotIdx"] = { keeper_log_info.last_snapshot_idx, "Index of the last log present in the last created snapshot." };
auto & keeper_connection_stats = keeper_dispatcher.getKeeperConnectionStats();
new_values["KeeperMinLatency"] = { keeper_connection_stats.getMinLatency(), "Minimal request latency of ClickHouse Keeper." };
new_values["KeeperMaxLatency"] = { keeper_connection_stats.getMaxLatency(), "Maximum request latency of ClickHouse Keeper." };
new_values["KeeperAvgLatency"] = { keeper_connection_stats.getAvgLatency(), "Average request latency of ClickHouse Keeper." };
new_values["KeeperPacketsReceived"] = { keeper_connection_stats.getPacketsReceived(), "Number of packets received by ClickHouse Keeper." };
new_values["KeeperPacketsSent"] = { keeper_connection_stats.getPacketsSent(), "Number of packets sent by ClickHouse Keeper." };
#endif
}
KeeperAsynchronousMetrics::KeeperAsynchronousMetrics(
TinyContextPtr tiny_context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_)
: AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_), tiny_context(std::move(tiny_context_))
{
}
void KeeperAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values, TimePoint /*update_time*/, TimePoint /*current_time*/)
{
#if USE_NURAFT
{
auto keeper_dispatcher = tiny_context->tryGetKeeperDispatcher();
if (keeper_dispatcher)
updateKeeperInformation(*keeper_dispatcher, new_values);
}
#endif
}
}

View File

@ -0,0 +1,25 @@
#pragma once
#include <Coordination/TinyContext.h>
#include <Common/AsynchronousMetrics.h>
namespace DB
{
class KeeperDispatcher;
void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousMetricValues & new_values);
class KeeperAsynchronousMetrics : public AsynchronousMetrics
{
public:
KeeperAsynchronousMetrics(
TinyContextPtr tiny_context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_);
private:
TinyContextPtr tiny_context;
void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) override;
};
}

View File

@ -109,7 +109,7 @@ uint64_t KeeperLogStore::size() const
void KeeperLogStore::end_of_append_batch(uint64_t /*start_index*/, uint64_t /*count*/) void KeeperLogStore::end_of_append_batch(uint64_t /*start_index*/, uint64_t /*count*/)
{ {
std::lock_guard lock(changelog_lock); std::lock_guard lock(changelog_lock);
changelog.flush(); changelog.flushAsync();
} }
nuraft::ptr<nuraft::log_entry> KeeperLogStore::getLatestConfigChange() const nuraft::ptr<nuraft::log_entry> KeeperLogStore::getLatestConfigChange() const
@ -132,4 +132,16 @@ bool KeeperLogStore::flushChangelogAndShutdown()
return true; return true;
} }
uint64_t KeeperLogStore::last_durable_index()
{
std::lock_guard lock(changelog_lock);
return changelog.lastDurableIndex();
}
void KeeperLogStore::setRaftServer(const nuraft::ptr<nuraft::raft_server> & raft_server)
{
std::lock_guard lock(changelog_lock);
return changelog.setRaftServer(raft_server);
}
} }

View File

@ -62,12 +62,16 @@ public:
/// Current log storage size /// Current log storage size
uint64_t size() const; uint64_t size() const;
uint64_t last_durable_index() override;
/// Flush batch of appended entries /// Flush batch of appended entries
void end_of_append_batch(uint64_t start_index, uint64_t count) override; void end_of_append_batch(uint64_t start_index, uint64_t count) override;
/// Get entry with latest config in logstore /// Get entry with latest config in logstore
nuraft::ptr<nuraft::log_entry> getLatestConfigChange() const; nuraft::ptr<nuraft::log_entry> getLatestConfigChange() const;
void setRaftServer(const nuraft::ptr<nuraft::raft_server> & raft_server);
private: private:
mutable std::mutex changelog_lock; mutable std::mutex changelog_lock;
Poco::Logger * log; Poco::Logger * log;

View File

@ -266,6 +266,7 @@ void KeeperServer::forceRecovery()
void KeeperServer::launchRaftServer(const Poco::Util::AbstractConfiguration & config, bool enable_ipv6) void KeeperServer::launchRaftServer(const Poco::Util::AbstractConfiguration & config, bool enable_ipv6)
{ {
nuraft::raft_params params; nuraft::raft_params params;
params.parallel_log_appending_ = true;
params.heart_beat_interval_ params.heart_beat_interval_
= getValueOrMaxInt32AndLogWarning(coordination_settings->heart_beat_interval_ms.totalMilliseconds(), "heart_beat_interval_ms", log); = getValueOrMaxInt32AndLogWarning(coordination_settings->heart_beat_interval_ms.totalMilliseconds(), "heart_beat_interval_ms", log);
params.election_timeout_lower_bound_ = getValueOrMaxInt32AndLogWarning( params.election_timeout_lower_bound_ = getValueOrMaxInt32AndLogWarning(
@ -352,6 +353,8 @@ void KeeperServer::launchRaftServer(const Poco::Util::AbstractConfiguration & co
if (!raft_instance) if (!raft_instance)
throw Exception(ErrorCodes::RAFT_ERROR, "Cannot allocate RAFT instance"); throw Exception(ErrorCodes::RAFT_ERROR, "Cannot allocate RAFT instance");
state_manager->getLogStore()->setRaftServer(raft_instance);
raft_instance->start_server(init_options.skip_initial_election_timeout_); raft_instance->start_server(init_options.skip_initial_election_timeout_);
nuraft::ptr<nuraft::raft_server> casted_raft_server = raft_instance; nuraft::ptr<nuraft::raft_server> casted_raft_server = raft_instance;
@ -446,8 +449,8 @@ void KeeperServer::shutdownRaftServer()
void KeeperServer::shutdown() void KeeperServer::shutdown()
{ {
state_manager->flushAndShutDownLogStore();
shutdownRaftServer(); shutdownRaftServer();
state_manager->flushAndShutDownLogStore();
state_machine->shutdownStorage(); state_machine->shutdownStorage();
} }

View File

@ -15,6 +15,7 @@
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
#include <Common/setThreadName.h> #include <Common/setThreadName.h>
#include <Common/LockMemoryExceptionInThread.h> #include <Common/LockMemoryExceptionInThread.h>
#include <Common/ProfileEvents.h>
#include <Coordination/pathUtils.h> #include <Coordination/pathUtils.h>
#include <Coordination/KeeperConstants.h> #include <Coordination/KeeperConstants.h>
@ -27,6 +28,19 @@
#include <base/defines.h> #include <base/defines.h>
#include <filesystem> #include <filesystem>
namespace ProfileEvents
{
extern const Event KeeperCreateRequest;
extern const Event KeeperRemoveRequest;
extern const Event KeeperSetRequest;
extern const Event KeeperCheckRequest;
extern const Event KeeperMultiRequest;
extern const Event KeeperMultiReadRequest;
extern const Event KeeperGetRequest;
extern const Event KeeperListRequest;
extern const Event KeeperExistsRequest;
}
namespace DB namespace DB
{ {
@ -865,6 +879,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr
std::vector<KeeperStorage::Delta> std::vector<KeeperStorage::Delta>
preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override
{ {
ProfileEvents::increment(ProfileEvents::KeeperCreateRequest);
Coordination::ZooKeeperCreateRequest & request = dynamic_cast<Coordination::ZooKeeperCreateRequest &>(*zk_request); Coordination::ZooKeeperCreateRequest & request = dynamic_cast<Coordination::ZooKeeperCreateRequest &>(*zk_request);
std::vector<KeeperStorage::Delta> new_deltas; std::vector<KeeperStorage::Delta> new_deltas;
@ -986,6 +1001,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce
std::vector<KeeperStorage::Delta> std::vector<KeeperStorage::Delta>
preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override
{ {
ProfileEvents::increment(ProfileEvents::KeeperGetRequest);
Coordination::ZooKeeperGetRequest & request = dynamic_cast<Coordination::ZooKeeperGetRequest &>(*zk_request); Coordination::ZooKeeperGetRequest & request = dynamic_cast<Coordination::ZooKeeperGetRequest &>(*zk_request);
if (request.path == Coordination::keeper_api_version_path) if (request.path == Coordination::keeper_api_version_path)
@ -1040,6 +1056,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce
Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override
{ {
ProfileEvents::increment(ProfileEvents::KeeperGetRequest);
return processImpl<true>(storage, zxid); return processImpl<true>(storage, zxid);
} }
}; };
@ -1055,6 +1072,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr
std::vector<KeeperStorage::Delta> std::vector<KeeperStorage::Delta>
preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override
{ {
ProfileEvents::increment(ProfileEvents::KeeperRemoveRequest);
Coordination::ZooKeeperRemoveRequest & request = dynamic_cast<Coordination::ZooKeeperRemoveRequest &>(*zk_request); Coordination::ZooKeeperRemoveRequest & request = dynamic_cast<Coordination::ZooKeeperRemoveRequest &>(*zk_request);
std::vector<KeeperStorage::Delta> new_deltas; std::vector<KeeperStorage::Delta> new_deltas;
@ -1145,6 +1163,7 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr
std::vector<KeeperStorage::Delta> std::vector<KeeperStorage::Delta>
preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override
{ {
ProfileEvents::increment(ProfileEvents::KeeperExistsRequest);
Coordination::ZooKeeperExistsRequest & request = dynamic_cast<Coordination::ZooKeeperExistsRequest &>(*zk_request); Coordination::ZooKeeperExistsRequest & request = dynamic_cast<Coordination::ZooKeeperExistsRequest &>(*zk_request);
if (!storage.uncommitted_state.getNode(request.path)) if (!storage.uncommitted_state.getNode(request.path))
@ -1194,6 +1213,7 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr
Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override
{ {
ProfileEvents::increment(ProfileEvents::KeeperExistsRequest);
return processImpl<true>(storage, zxid); return processImpl<true>(storage, zxid);
} }
}; };
@ -1209,6 +1229,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce
std::vector<KeeperStorage::Delta> std::vector<KeeperStorage::Delta>
preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override
{ {
ProfileEvents::increment(ProfileEvents::KeeperSetRequest);
Coordination::ZooKeeperSetRequest & request = dynamic_cast<Coordination::ZooKeeperSetRequest &>(*zk_request); Coordination::ZooKeeperSetRequest & request = dynamic_cast<Coordination::ZooKeeperSetRequest &>(*zk_request);
std::vector<KeeperStorage::Delta> new_deltas; std::vector<KeeperStorage::Delta> new_deltas;
@ -1301,6 +1322,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc
std::vector<KeeperStorage::Delta> std::vector<KeeperStorage::Delta>
preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override
{ {
ProfileEvents::increment(ProfileEvents::KeeperListRequest);
Coordination::ZooKeeperListRequest & request = dynamic_cast<Coordination::ZooKeeperListRequest &>(*zk_request); Coordination::ZooKeeperListRequest & request = dynamic_cast<Coordination::ZooKeeperListRequest &>(*zk_request);
if (!storage.uncommitted_state.getNode(request.path)) if (!storage.uncommitted_state.getNode(request.path))
@ -1387,6 +1409,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc
Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override
{ {
ProfileEvents::increment(ProfileEvents::KeeperListRequest);
return processImpl<true>(storage, zxid); return processImpl<true>(storage, zxid);
} }
}; };
@ -1402,6 +1425,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro
std::vector<KeeperStorage::Delta> std::vector<KeeperStorage::Delta>
preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override
{ {
ProfileEvents::increment(ProfileEvents::KeeperCheckRequest);
Coordination::ZooKeeperCheckRequest & request = dynamic_cast<Coordination::ZooKeeperCheckRequest &>(*zk_request); Coordination::ZooKeeperCheckRequest & request = dynamic_cast<Coordination::ZooKeeperCheckRequest &>(*zk_request);
if (!storage.uncommitted_state.getNode(request.path)) if (!storage.uncommitted_state.getNode(request.path))
@ -1463,6 +1487,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro
Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override
{ {
ProfileEvents::increment(ProfileEvents::KeeperCheckRequest);
return processImpl<true>(storage, zxid); return processImpl<true>(storage, zxid);
} }
}; };
@ -1689,6 +1714,7 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro
std::vector<KeeperStorage::Delta> std::vector<KeeperStorage::Delta>
preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override
{ {
ProfileEvents::increment(ProfileEvents::KeeperMultiRequest);
std::vector<Coordination::Error> response_errors; std::vector<Coordination::Error> response_errors;
response_errors.reserve(concrete_requests.size()); response_errors.reserve(concrete_requests.size());
uint64_t current_digest = digest; uint64_t current_digest = digest;
@ -1756,6 +1782,7 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro
Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override
{ {
ProfileEvents::increment(ProfileEvents::KeeperMultiReadRequest);
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
Coordination::ZooKeeperMultiResponse & response = dynamic_cast<Coordination::ZooKeeperMultiResponse &>(*response_ptr); Coordination::ZooKeeperMultiResponse & response = dynamic_cast<Coordination::ZooKeeperMultiResponse &>(*response_ptr);

View File

@ -1,4 +1,4 @@
#include "TinyContext.h" #include <Coordination/TinyContext.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Coordination/KeeperDispatcher.h> #include <Coordination/KeeperDispatcher.h>

View File

@ -31,4 +31,6 @@ private:
ConfigurationPtr config TSA_GUARDED_BY(keeper_dispatcher_mutex); ConfigurationPtr config TSA_GUARDED_BY(keeper_dispatcher_mutex);
}; };
using TinyContextPtr = std::shared_ptr<TinyContext>;
} }

View File

@ -67,6 +67,7 @@ class CoordinationTest : public ::testing::TestWithParam<CompressionParam>
{ {
protected: protected:
DB::KeeperContextPtr keeper_context = std::make_shared<DB::KeeperContext>(); DB::KeeperContextPtr keeper_context = std::make_shared<DB::KeeperContext>();
Poco::Logger * log{&Poco::Logger::get("CoordinationTest")};
}; };
TEST_P(CoordinationTest, BuildTest) TEST_P(CoordinationTest, BuildTest)
@ -129,10 +130,13 @@ struct SimpliestRaftServer
params.snapshot_distance_ = 1; /// forcefully send snapshots params.snapshot_distance_ = 1; /// forcefully send snapshots
params.client_req_timeout_ = 3000; params.client_req_timeout_ = 3000;
params.return_method_ = nuraft::raft_params::blocking; params.return_method_ = nuraft::raft_params::blocking;
params.parallel_log_appending_ = true;
nuraft::raft_server::init_options opts;
opts.start_server_in_constructor_ = false;
raft_instance = launcher.init( raft_instance = launcher.init(
state_machine, state_manager, nuraft::cs_new<DB::LoggerWrapper>("ToyRaftLogger", DB::LogsLevel::trace), port, state_machine, state_manager, nuraft::cs_new<DB::LoggerWrapper>("ToyRaftLogger", DB::LogsLevel::trace), port,
nuraft::asio_service::options{}, params); nuraft::asio_service::options{}, params, opts);
if (!raft_instance) if (!raft_instance)
{ {
@ -140,6 +144,10 @@ struct SimpliestRaftServer
_exit(1); _exit(1);
} }
state_manager->getLogStore()->setRaftServer(raft_instance);
raft_instance->start_server(false);
std::cout << "init Raft instance " << server_id; std::cout << "init Raft instance " << server_id;
for (size_t ii = 0; ii < 20; ++ii) for (size_t ii = 0; ii < 20; ++ii)
{ {
@ -207,7 +215,7 @@ TEST_P(CoordinationTest, TestSummingRaft1)
while (s1.state_machine->getValue() != 143) while (s1.state_machine->getValue() != 143)
{ {
std::cout << "Waiting s1 to apply entry\n"; LOG_INFO(log, "Waiting s1 to apply entry");
std::this_thread::sleep_for(std::chrono::milliseconds(100)); std::this_thread::sleep_for(std::chrono::milliseconds(100));
} }
@ -240,6 +248,15 @@ TEST_P(CoordinationTest, ChangelogTestSimple)
EXPECT_EQ(changelog.log_entries(1, 2)->size(), 1); EXPECT_EQ(changelog.log_entries(1, 2)->size(), 1);
} }
namespace
{
void waitDurableLogs(nuraft::log_store & log_store)
{
while (log_store.last_durable_index() != log_store.next_slot() - 1)
std::this_thread::sleep_for(std::chrono::milliseconds(200));
}
}
TEST_P(CoordinationTest, ChangelogTestFile) TEST_P(CoordinationTest, ChangelogTestFile)
{ {
@ -250,6 +267,9 @@ TEST_P(CoordinationTest, ChangelogTestFile)
auto entry = getLogEntry("hello world", 77); auto entry = getLogEntry("hello world", 77);
changelog.append(entry); changelog.append(entry);
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
waitDurableLogs(changelog);
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension));
for (const auto & p : fs::directory_iterator("./logs")) for (const auto & p : fs::directory_iterator("./logs"))
EXPECT_EQ(p.path(), "./logs/changelog_1_5.bin" + params.extension); EXPECT_EQ(p.path(), "./logs/changelog_1_5.bin" + params.extension);
@ -261,6 +281,8 @@ TEST_P(CoordinationTest, ChangelogTestFile)
changelog.append(entry); changelog.append(entry);
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
waitDurableLogs(changelog);
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension));
} }
@ -271,6 +293,7 @@ TEST_P(CoordinationTest, ChangelogReadWrite)
ChangelogDirTest test("./logs"); ChangelogDirTest test("./logs");
DB::KeeperLogStore changelog("./logs", 1000, true, params.enable_compression); DB::KeeperLogStore changelog("./logs", 1000, true, params.enable_compression);
changelog.init(1, 0); changelog.init(1, 0);
for (size_t i = 0; i < 10; ++i) for (size_t i = 0; i < 10; ++i)
{ {
auto entry = getLogEntry("hello world", i * 10); auto entry = getLogEntry("hello world", i * 10);
@ -280,6 +303,8 @@ TEST_P(CoordinationTest, ChangelogReadWrite)
EXPECT_EQ(changelog.size(), 10); EXPECT_EQ(changelog.size(), 10);
waitDurableLogs(changelog);
DB::KeeperLogStore changelog_reader("./logs", 1000, true, params.enable_compression); DB::KeeperLogStore changelog_reader("./logs", 1000, true, params.enable_compression);
changelog_reader.init(1, 0); changelog_reader.init(1, 0);
EXPECT_EQ(changelog_reader.size(), 10); EXPECT_EQ(changelog_reader.size(), 10);
@ -315,6 +340,8 @@ TEST_P(CoordinationTest, ChangelogWriteAt)
changelog.write_at(7, entry); changelog.write_at(7, entry);
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
waitDurableLogs(changelog);
EXPECT_EQ(changelog.size(), 7); EXPECT_EQ(changelog.size(), 7);
EXPECT_EQ(changelog.last_entry()->get_term(), 77); EXPECT_EQ(changelog.last_entry()->get_term(), 77);
EXPECT_EQ(changelog.entry_at(7)->get_term(), 77); EXPECT_EQ(changelog.entry_at(7)->get_term(), 77);
@ -344,6 +371,9 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead)
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
EXPECT_EQ(changelog.size(), 7); EXPECT_EQ(changelog.size(), 7);
waitDurableLogs(changelog);
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension));
@ -358,6 +388,8 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead)
} }
changelog_reader.end_of_append_batch(0, 0); changelog_reader.end_of_append_batch(0, 0);
EXPECT_EQ(changelog_reader.size(), 10); EXPECT_EQ(changelog_reader.size(), 10);
waitDurableLogs(changelog_reader);
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension));
@ -371,6 +403,8 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead)
changelog_reader.append(entry); changelog_reader.append(entry);
changelog_reader.end_of_append_batch(0, 0); changelog_reader.end_of_append_batch(0, 0);
EXPECT_EQ(changelog_reader.size(), 11); EXPECT_EQ(changelog_reader.size(), 11);
waitDurableLogs(changelog_reader);
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension));
@ -396,6 +430,8 @@ TEST_P(CoordinationTest, ChangelogTestCompaction)
} }
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
waitDurableLogs(changelog);
EXPECT_EQ(changelog.size(), 3); EXPECT_EQ(changelog.size(), 3);
changelog.compact(2); changelog.compact(2);
@ -416,6 +452,8 @@ TEST_P(CoordinationTest, ChangelogTestCompaction)
changelog.append(e4); changelog.append(e4);
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
waitDurableLogs(changelog);
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension));
@ -454,6 +492,8 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperations)
EXPECT_EQ(changelog.size(), 10); EXPECT_EQ(changelog.size(), 10);
waitDurableLogs(changelog);
auto entries = changelog.pack(1, 5); auto entries = changelog.pack(1, 5);
DB::KeeperLogStore apply_changelog("./logs", 100, true, params.enable_compression); DB::KeeperLogStore apply_changelog("./logs", 100, true, params.enable_compression);
@ -499,6 +539,8 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperationsEmpty)
EXPECT_EQ(changelog.size(), 10); EXPECT_EQ(changelog.size(), 10);
waitDurableLogs(changelog);
auto entries = changelog.pack(5, 5); auto entries = changelog.pack(5, 5);
ChangelogDirTest test1("./logs1"); ChangelogDirTest test1("./logs1");
@ -543,6 +585,8 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtPreviousFile)
} }
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
waitDurableLogs(changelog);
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension));
@ -561,6 +605,8 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtPreviousFile)
EXPECT_EQ(changelog.next_slot(), 8); EXPECT_EQ(changelog.next_slot(), 8);
EXPECT_EQ(changelog.last_entry()->get_term(), 5555); EXPECT_EQ(changelog.last_entry()->get_term(), 5555);
waitDurableLogs(changelog);
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension));
@ -592,6 +638,8 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtFileBorder)
} }
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
waitDurableLogs(changelog);
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension));
@ -610,6 +658,8 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtFileBorder)
EXPECT_EQ(changelog.next_slot(), 12); EXPECT_EQ(changelog.next_slot(), 12);
EXPECT_EQ(changelog.last_entry()->get_term(), 5555); EXPECT_EQ(changelog.last_entry()->get_term(), 5555);
waitDurableLogs(changelog);
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension));
@ -633,7 +683,6 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtAllFiles)
ChangelogDirTest test("./logs"); ChangelogDirTest test("./logs");
DB::KeeperLogStore changelog("./logs", 5, true, params.enable_compression); DB::KeeperLogStore changelog("./logs", 5, true, params.enable_compression);
changelog.init(1, 0); changelog.init(1, 0);
for (size_t i = 0; i < 33; ++i) for (size_t i = 0; i < 33; ++i)
{ {
auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10);
@ -641,6 +690,8 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtAllFiles)
} }
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
waitDurableLogs(changelog);
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension));
@ -659,6 +710,8 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtAllFiles)
EXPECT_EQ(changelog.next_slot(), 2); EXPECT_EQ(changelog.next_slot(), 2);
EXPECT_EQ(changelog.last_entry()->get_term(), 5555); EXPECT_EQ(changelog.last_entry()->get_term(), 5555);
waitDurableLogs(changelog);
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension));
EXPECT_FALSE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_FALSE(fs::exists("./logs/changelog_6_10.bin" + params.extension));
@ -683,6 +736,8 @@ TEST_P(CoordinationTest, ChangelogTestStartNewLogAfterRead)
} }
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
EXPECT_EQ(changelog.size(), 35); EXPECT_EQ(changelog.size(), 35);
waitDurableLogs(changelog);
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension));
@ -692,7 +747,6 @@ TEST_P(CoordinationTest, ChangelogTestStartNewLogAfterRead)
EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin" + params.extension));
EXPECT_FALSE(fs::exists("./logs/changelog_36_40.bin" + params.extension)); EXPECT_FALSE(fs::exists("./logs/changelog_36_40.bin" + params.extension));
DB::KeeperLogStore changelog_reader("./logs", 5, true, params.enable_compression); DB::KeeperLogStore changelog_reader("./logs", 5, true, params.enable_compression);
changelog_reader.init(1, 0); changelog_reader.init(1, 0);
@ -701,6 +755,8 @@ TEST_P(CoordinationTest, ChangelogTestStartNewLogAfterRead)
changelog_reader.end_of_append_batch(0, 0); changelog_reader.end_of_append_batch(0, 0);
EXPECT_EQ(changelog_reader.size(), 36); EXPECT_EQ(changelog_reader.size(), 36);
waitDurableLogs(changelog_reader);
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension));
@ -746,6 +802,8 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate)
} }
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
EXPECT_EQ(changelog.size(), 35); EXPECT_EQ(changelog.size(), 35);
waitDurableLogs(changelog);
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension));
@ -779,6 +837,8 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate)
EXPECT_EQ(changelog_reader.size(), 11); EXPECT_EQ(changelog_reader.size(), 11);
EXPECT_EQ(changelog_reader.last_entry()->get_term(), 7777); EXPECT_EQ(changelog_reader.last_entry()->get_term(), 7777);
waitDurableLogs(changelog_reader);
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension));
@ -809,6 +869,7 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2)
} }
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
waitDurableLogs(changelog);
EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_21_40.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_21_40.bin" + params.extension));
@ -824,6 +885,9 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2)
auto entry = getLogEntry("hello_world", 7777); auto entry = getLogEntry("hello_world", 7777);
changelog_reader.append(entry); changelog_reader.append(entry);
changelog_reader.end_of_append_batch(0, 0); changelog_reader.end_of_append_batch(0, 0);
waitDurableLogs(changelog_reader);
EXPECT_EQ(changelog_reader.size(), 1); EXPECT_EQ(changelog_reader.size(), 1);
EXPECT_EQ(changelog_reader.last_entry()->get_term(), 7777); EXPECT_EQ(changelog_reader.last_entry()->get_term(), 7777);
@ -848,6 +912,7 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles)
} }
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
waitDurableLogs(changelog);
EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_21_40.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_21_40.bin" + params.extension));
@ -874,6 +939,8 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles2)
} }
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
waitDurableLogs(changelog);
EXPECT_TRUE(fs::exists("./logs/changelog_1_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_1_10.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_11_20.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_20.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_21_30.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_21_30.bin" + params.extension));
@ -1330,6 +1397,8 @@ void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint
changelog.append(entry); changelog.append(entry);
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
waitDurableLogs(changelog);
state_machine->pre_commit(i, changelog.entry_at(i)->get_buf()); state_machine->pre_commit(i, changelog.entry_at(i)->get_buf());
state_machine->commit(i, changelog.entry_at(i)->get_buf()); state_machine->commit(i, changelog.entry_at(i)->get_buf());
bool snapshot_created = false; bool snapshot_created = false;
@ -1339,7 +1408,7 @@ void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint
nuraft::async_result<bool>::handler_type when_done = [&snapshot_created] (bool & ret, nuraft::ptr<std::exception> &/*exception*/) nuraft::async_result<bool>::handler_type when_done = [&snapshot_created] (bool & ret, nuraft::ptr<std::exception> &/*exception*/)
{ {
snapshot_created = ret; snapshot_created = ret;
std::cerr << "Snapshot finished\n"; LOG_INFO(&Poco::Logger::get("CoordinationTest"), "Snapshot finished");
}; };
state_machine->create_snapshot(s, when_done); state_machine->create_snapshot(s, when_done);
@ -1511,6 +1580,8 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges)
changelog.append(entry); changelog.append(entry);
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
} }
waitDurableLogs(changelog);
} }
@ -1527,6 +1598,8 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges)
changelog_1.end_of_append_batch(0, 0); changelog_1.end_of_append_batch(0, 0);
} }
waitDurableLogs(changelog_1);
EXPECT_TRUE(fs::exists("./logs/changelog_1_100.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_1_100.bin" + params.extension));
EXPECT_TRUE(fs::exists("./logs/changelog_101_110.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_101_110.bin" + params.extension));
@ -1542,6 +1615,8 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges)
changelog_2.end_of_append_batch(0, 0); changelog_2.end_of_append_batch(0, 0);
} }
waitDurableLogs(changelog_2);
changelog_2.compact(105); changelog_2.compact(105);
std::this_thread::sleep_for(std::chrono::microseconds(1000)); std::this_thread::sleep_for(std::chrono::microseconds(1000));
@ -1562,6 +1637,8 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges)
changelog_3.end_of_append_batch(0, 0); changelog_3.end_of_append_batch(0, 0);
} }
waitDurableLogs(changelog_3);
changelog_3.compact(125); changelog_3.compact(125);
std::this_thread::sleep_for(std::chrono::microseconds(1000)); std::this_thread::sleep_for(std::chrono::microseconds(1000));
EXPECT_FALSE(fs::exists("./logs/changelog_101_110.bin" + params.extension)); EXPECT_FALSE(fs::exists("./logs/changelog_101_110.bin" + params.extension));
@ -1609,6 +1686,7 @@ TEST_P(CoordinationTest, TestCompressedLogsMultipleRewrite)
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
} }
waitDurableLogs(changelog);
DB::KeeperLogStore changelog1("./logs", 100, true, test_params.enable_compression); DB::KeeperLogStore changelog1("./logs", 100, true, test_params.enable_compression);
changelog1.init(0, 3); changelog1.init(0, 3);
@ -1683,43 +1761,47 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth)
auto params = GetParam(); auto params = GetParam();
ChangelogDirTest test("./logs"); ChangelogDirTest test("./logs");
{ {
std::cerr << "================First time=====================\n"; LOG_INFO(log, "================First time=====================");
DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression); DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression);
changelog.init(1, 0); changelog.init(1, 0);
auto entry = getLogEntry("hello_world", 1000); auto entry = getLogEntry("hello_world", 1000);
changelog.append(entry); changelog.append(entry);
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
EXPECT_EQ(changelog.next_slot(), 2); EXPECT_EQ(changelog.next_slot(), 2);
waitDurableLogs(changelog);
} }
{ {
std::cerr << "================Second time=====================\n"; LOG_INFO(log, "================Second time=====================");
DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression); DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression);
changelog.init(1, 0); changelog.init(1, 0);
auto entry = getLogEntry("hello_world", 1000); auto entry = getLogEntry("hello_world", 1000);
changelog.append(entry); changelog.append(entry);
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
EXPECT_EQ(changelog.next_slot(), 3); EXPECT_EQ(changelog.next_slot(), 3);
waitDurableLogs(changelog);
} }
{ {
std::cerr << "================Third time=====================\n"; LOG_INFO(log, "================Third time=====================");
DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression); DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression);
changelog.init(1, 0); changelog.init(1, 0);
auto entry = getLogEntry("hello_world", 1000); auto entry = getLogEntry("hello_world", 1000);
changelog.append(entry); changelog.append(entry);
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
EXPECT_EQ(changelog.next_slot(), 4); EXPECT_EQ(changelog.next_slot(), 4);
waitDurableLogs(changelog);
} }
{ {
std::cerr << "================Fourth time=====================\n"; LOG_INFO(log, "================Fourth time=====================");
DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression); DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression);
changelog.init(1, 0); changelog.init(1, 0);
auto entry = getLogEntry("hello_world", 1000); auto entry = getLogEntry("hello_world", 1000);
changelog.append(entry); changelog.append(entry);
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
EXPECT_EQ(changelog.next_slot(), 5); EXPECT_EQ(changelog.next_slot(), 5);
waitDurableLogs(changelog);
} }
} }
@ -1730,7 +1812,7 @@ TEST_P(CoordinationTest, ChangelogInsertMultipleTimesSmooth)
ChangelogDirTest test("./logs"); ChangelogDirTest test("./logs");
for (size_t i = 0; i < 36; ++i) for (size_t i = 0; i < 36; ++i)
{ {
std::cerr << "================First time=====================\n"; LOG_INFO(log, "================First time=====================");
DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression); DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression);
changelog.init(1, 0); changelog.init(1, 0);
for (size_t j = 0; j < 7; ++j) for (size_t j = 0; j < 7; ++j)
@ -1739,6 +1821,7 @@ TEST_P(CoordinationTest, ChangelogInsertMultipleTimesSmooth)
changelog.append(entry); changelog.append(entry);
} }
changelog.end_of_append_batch(0, 0); changelog.end_of_append_batch(0, 0);
waitDurableLogs(changelog);
} }
DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression); DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression);
@ -1750,37 +1833,49 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard)
{ {
auto params = GetParam(); auto params = GetParam();
ChangelogDirTest test("./logs"); ChangelogDirTest test("./logs");
std::cerr << "================First time=====================\n"; {
LOG_INFO(log, "================First time=====================");
DB::KeeperLogStore changelog1("./logs", 100, true, params.enable_compression); DB::KeeperLogStore changelog1("./logs", 100, true, params.enable_compression);
changelog1.init(1, 0); changelog1.init(1, 0);
auto entry = getLogEntry("hello_world", 1000); auto entry = getLogEntry("hello_world", 1000);
changelog1.append(entry); changelog1.append(entry);
changelog1.end_of_append_batch(0, 0); changelog1.end_of_append_batch(0, 0);
EXPECT_EQ(changelog1.next_slot(), 2); EXPECT_EQ(changelog1.next_slot(), 2);
waitDurableLogs(changelog1);
}
std::cerr << "================Second time=====================\n"; {
LOG_INFO(log, "================Second time=====================");
DB::KeeperLogStore changelog2("./logs", 100, true, params.enable_compression); DB::KeeperLogStore changelog2("./logs", 100, true, params.enable_compression);
changelog2.init(1, 0); changelog2.init(1, 0);
entry = getLogEntry("hello_world", 1000); auto entry = getLogEntry("hello_world", 1000);
changelog2.append(entry); changelog2.append(entry);
changelog2.end_of_append_batch(0, 0); changelog2.end_of_append_batch(0, 0);
EXPECT_EQ(changelog2.next_slot(), 3); EXPECT_EQ(changelog2.next_slot(), 3);
waitDurableLogs(changelog2);
}
std::cerr << "================Third time=====================\n"; {
LOG_INFO(log, "================Third time=====================");
DB::KeeperLogStore changelog3("./logs", 100, true, params.enable_compression); DB::KeeperLogStore changelog3("./logs", 100, true, params.enable_compression);
changelog3.init(1, 0); changelog3.init(1, 0);
entry = getLogEntry("hello_world", 1000); auto entry = getLogEntry("hello_world", 1000);
changelog3.append(entry); changelog3.append(entry);
changelog3.end_of_append_batch(0, 0); changelog3.end_of_append_batch(0, 0);
EXPECT_EQ(changelog3.next_slot(), 4); EXPECT_EQ(changelog3.next_slot(), 4);
waitDurableLogs(changelog3);
}
std::cerr << "================Fourth time=====================\n"; {
LOG_INFO(log, "================Fourth time=====================");
DB::KeeperLogStore changelog4("./logs", 100, true, params.enable_compression); DB::KeeperLogStore changelog4("./logs", 100, true, params.enable_compression);
changelog4.init(1, 0); changelog4.init(1, 0);
entry = getLogEntry("hello_world", 1000); auto entry = getLogEntry("hello_world", 1000);
changelog4.append(entry); changelog4.append(entry);
changelog4.end_of_append_batch(0, 0); changelog4.end_of_append_batch(0, 0);
EXPECT_EQ(changelog4.next_slot(), 5); EXPECT_EQ(changelog4.next_slot(), 5);
waitDurableLogs(changelog4);
}
} }
TEST_P(CoordinationTest, TestStorageSnapshotEqual) TEST_P(CoordinationTest, TestStorageSnapshotEqual)

View File

@ -398,6 +398,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \ M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \
M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \ M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \
M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \
M(Bool, trace_profile_events, false, "Send to system.trace_log profile event and value of increment on each increment with 'ProfileEvent' trace_type", 0) \
\ \
M(UInt64, memory_usage_overcommit_max_wait_microseconds, 5'000'000, "Maximum time thread will wait for memory to be freed in the case of memory overcommit. If timeout is reached and memory is not freed, exception is thrown.", 0) \ M(UInt64, memory_usage_overcommit_max_wait_microseconds, 5'000'000, "Maximum time thread will wait for memory to be freed in the case of memory overcommit. If timeout is reached and memory is not freed, exception is thrown.", 0) \
\ \
@ -782,6 +783,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
M(Bool, input_format_values_accurate_types_of_literals, true, "For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.", 0) \ M(Bool, input_format_values_accurate_types_of_literals, true, "For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.", 0) \
M(Bool, input_format_avro_allow_missing_fields, false, "For Avro/AvroConfluent format: when field is not found in schema use default value instead of error", 0) \ M(Bool, input_format_avro_allow_missing_fields, false, "For Avro/AvroConfluent format: when field is not found in schema use default value instead of error", 0) \
M(Bool, input_format_avro_null_as_default, false, "For Avro/AvroConfluent format: insert default in case of null and non Nullable column", 0) \ M(Bool, input_format_avro_null_as_default, false, "For Avro/AvroConfluent format: insert default in case of null and non Nullable column", 0) \
M(UInt64, format_binary_max_string_size, 1_GiB, "The maximum allowed size for String in RowBinary format. It prevents allocating large amount of memory in case of corrupted data. 0 means there is no limit", 0) \
M(URI, format_avro_schema_registry_url, "", "For AvroConfluent format: Confluent Schema Registry URL.", 0) \ M(URI, format_avro_schema_registry_url, "", "For AvroConfluent format: Confluent Schema Registry URL.", 0) \
\ \
M(Bool, output_format_json_quote_64bit_integers, true, "Controls quoting of 64-bit integers in JSON output format.", 0) \ M(Bool, output_format_json_quote_64bit_integers, true, "Controls quoting of 64-bit integers in JSON output format.", 0) \

View File

@ -78,6 +78,7 @@ namespace SettingsChangesHistory
/// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972)
static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> settings_changes_history = static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> settings_changes_history =
{ {
{"22.12", {{"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}},
{"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}},
{"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}},
{"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"},

View File

@ -303,17 +303,17 @@ public:
*/ */
/// There is two variants for binary serde. First variant work with Field. /// There is two variants for binary serde. First variant work with Field.
virtual void serializeBinary(const Field & field, WriteBuffer & ostr) const = 0; virtual void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const = 0;
virtual void deserializeBinary(Field & field, ReadBuffer & istr) const = 0; virtual void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const = 0;
/// Other variants takes a column, to avoid creating temporary Field object. /// Other variants takes a column, to avoid creating temporary Field object.
/// Column must be non-constant. /// Column must be non-constant.
/// Serialize one value of a column at specified row number. /// Serialize one value of a column at specified row number.
virtual void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const = 0; virtual void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
/// Deserialize one value and insert into a column. /// Deserialize one value and insert into a column.
/// If method will throw an exception, then column will be in same state as before call to method. /// If method will throw an exception, then column will be in same state as before call to method.
virtual void deserializeBinary(IColumn & column, ReadBuffer & istr) const = 0; virtual void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
/** Text serialization with escaping but without quoting. /** Text serialization with escaping but without quoting.
*/ */

View File

@ -17,13 +17,13 @@
namespace DB namespace DB
{ {
void SerializationAggregateFunction::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationAggregateFunction::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const
{ {
const AggregateFunctionStateData & state = field.get<const AggregateFunctionStateData &>(); const AggregateFunctionStateData & state = field.get<const AggregateFunctionStateData &>();
writeBinary(state.data, ostr); writeBinary(state.data, ostr);
} }
void SerializationAggregateFunction::deserializeBinary(Field & field, ReadBuffer & istr) const void SerializationAggregateFunction::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const
{ {
field = AggregateFunctionStateData(); field = AggregateFunctionStateData();
AggregateFunctionStateData & s = field.get<AggregateFunctionStateData &>(); AggregateFunctionStateData & s = field.get<AggregateFunctionStateData &>();
@ -31,12 +31,12 @@ void SerializationAggregateFunction::deserializeBinary(Field & field, ReadBuffer
s.name = type_name; s.name = type_name;
} }
void SerializationAggregateFunction::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const void SerializationAggregateFunction::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
{ {
function->serialize(assert_cast<const ColumnAggregateFunction &>(column).getData()[row_num], ostr, version); function->serialize(assert_cast<const ColumnAggregateFunction &>(column).getData()[row_num], ostr, version);
} }
void SerializationAggregateFunction::deserializeBinary(IColumn & column, ReadBuffer & istr) const void SerializationAggregateFunction::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{ {
ColumnAggregateFunction & column_concrete = assert_cast<ColumnAggregateFunction &>(column); ColumnAggregateFunction & column_concrete = assert_cast<ColumnAggregateFunction &>(column);

View File

@ -22,11 +22,11 @@ public:
: function(function_), type_name(std::move(type_name_)), version(version_) {} : function(function_), type_name(std::move(type_name_)), version(version_) {}
/// NOTE These two functions for serializing single values are incompatible with the functions below. /// NOTE These two functions for serializing single values are incompatible with the functions below.
void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override; void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override;
void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override;
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;

View File

@ -27,18 +27,18 @@ static constexpr size_t MAX_ARRAY_SIZE = 1ULL << 30;
static constexpr size_t MAX_ARRAYS_SIZE = 1ULL << 40; static constexpr size_t MAX_ARRAYS_SIZE = 1ULL << 40;
void SerializationArray::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationArray::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const
{ {
const Array & a = field.get<const Array &>(); const Array & a = field.get<const Array &>();
writeVarUInt(a.size(), ostr); writeVarUInt(a.size(), ostr);
for (size_t i = 0; i < a.size(); ++i) for (size_t i = 0; i < a.size(); ++i)
{ {
nested->serializeBinary(a[i], ostr); nested->serializeBinary(a[i], ostr, settings);
} }
} }
void SerializationArray::deserializeBinary(Field & field, ReadBuffer & istr) const void SerializationArray::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const
{ {
size_t size; size_t size;
readVarUInt(size, istr); readVarUInt(size, istr);
@ -46,11 +46,11 @@ void SerializationArray::deserializeBinary(Field & field, ReadBuffer & istr) con
Array & arr = field.get<Array &>(); Array & arr = field.get<Array &>();
arr.reserve(size); arr.reserve(size);
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
nested->deserializeBinary(arr.emplace_back(), istr); nested->deserializeBinary(arr.emplace_back(), istr, settings);
} }
void SerializationArray::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const void SerializationArray::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{ {
const ColumnArray & column_array = assert_cast<const ColumnArray &>(column); const ColumnArray & column_array = assert_cast<const ColumnArray &>(column);
const ColumnArray::Offsets & offsets = column_array.getOffsets(); const ColumnArray::Offsets & offsets = column_array.getOffsets();
@ -63,11 +63,11 @@ void SerializationArray::serializeBinary(const IColumn & column, size_t row_num,
const IColumn & nested_column = column_array.getData(); const IColumn & nested_column = column_array.getData();
for (size_t i = offset; i < next_offset; ++i) for (size_t i = offset; i < next_offset; ++i)
nested->serializeBinary(nested_column, i, ostr); nested->serializeBinary(nested_column, i, ostr, settings);
} }
void SerializationArray::deserializeBinary(IColumn & column, ReadBuffer & istr) const void SerializationArray::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{ {
ColumnArray & column_array = assert_cast<ColumnArray &>(column); ColumnArray & column_array = assert_cast<ColumnArray &>(column);
ColumnArray::Offsets & offsets = column_array.getOffsets(); ColumnArray::Offsets & offsets = column_array.getOffsets();
@ -81,7 +81,7 @@ void SerializationArray::deserializeBinary(IColumn & column, ReadBuffer & istr)
try try
{ {
for (; i < size; ++i) for (; i < size; ++i)
nested->deserializeBinary(nested_column, istr); nested->deserializeBinary(nested_column, istr, settings);
} }
catch (...) catch (...)
{ {

View File

@ -13,10 +13,10 @@ private:
public: public:
explicit SerializationArray(const SerializationPtr & nested_) : nested(nested_) {} explicit SerializationArray(const SerializationPtr & nested_) : nested(nested_) {}
void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override; void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override; void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override;

View File

@ -12,14 +12,14 @@ namespace DB
{ {
template <typename T> template <typename T>
void SerializationDecimalBase<T>::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationDecimalBase<T>::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const
{ {
FieldType x = field.get<DecimalField<T>>(); FieldType x = field.get<DecimalField<T>>();
writeBinary(x, ostr); writeBinary(x, ostr);
} }
template <typename T> template <typename T>
void SerializationDecimalBase<T>::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const void SerializationDecimalBase<T>::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
{ {
const FieldType & x = assert_cast<const ColumnType &>(column).getElement(row_num); const FieldType & x = assert_cast<const ColumnType &>(column).getElement(row_num);
writeBinary(x, ostr); writeBinary(x, ostr);
@ -39,7 +39,7 @@ void SerializationDecimalBase<T>::serializeBinaryBulk(const IColumn & column, Wr
} }
template <typename T> template <typename T>
void SerializationDecimalBase<T>::deserializeBinary(Field & field, ReadBuffer & istr) const void SerializationDecimalBase<T>::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const
{ {
typename FieldType::NativeType x; typename FieldType::NativeType x;
readBinary(x, istr); readBinary(x, istr);
@ -47,7 +47,7 @@ void SerializationDecimalBase<T>::deserializeBinary(Field & field, ReadBuffer &
} }
template <typename T> template <typename T>
void SerializationDecimalBase<T>::deserializeBinary(IColumn & column, ReadBuffer & istr) const void SerializationDecimalBase<T>::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{ {
typename FieldType::NativeType x; typename FieldType::NativeType x;
readBinary(x, istr); readBinary(x, istr);

View File

@ -20,12 +20,12 @@ public:
SerializationDecimalBase(UInt32 precision_, UInt32 scale_) SerializationDecimalBase(UInt32 precision_, UInt32 scale_)
: precision(precision_), scale(scale_) {} : precision(precision_), scale(scale_) {}
void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override; void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override;
}; };

View File

@ -26,7 +26,7 @@ namespace ErrorCodes
static constexpr size_t MAX_STRINGS_SIZE = 1ULL << 30; static constexpr size_t MAX_STRINGS_SIZE = 1ULL << 30;
void SerializationFixedString::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationFixedString::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const
{ {
const String & s = field.get<const String &>(); const String & s = field.get<const String &>();
ostr.write(s.data(), std::min(s.size(), n)); ostr.write(s.data(), std::min(s.size(), n));
@ -36,7 +36,7 @@ void SerializationFixedString::serializeBinary(const Field & field, WriteBuffer
} }
void SerializationFixedString::deserializeBinary(Field & field, ReadBuffer & istr) const void SerializationFixedString::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const
{ {
field = String(); field = String();
String & s = field.get<String &>(); String & s = field.get<String &>();
@ -45,13 +45,13 @@ void SerializationFixedString::deserializeBinary(Field & field, ReadBuffer & ist
} }
void SerializationFixedString::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const void SerializationFixedString::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
{ {
ostr.write(reinterpret_cast<const char *>(&assert_cast<const ColumnFixedString &>(column).getChars()[n * row_num]), n); ostr.write(reinterpret_cast<const char *>(&assert_cast<const ColumnFixedString &>(column).getChars()[n * row_num]), n);
} }
void SerializationFixedString::deserializeBinary(IColumn & column, ReadBuffer & istr) const void SerializationFixedString::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{ {
ColumnFixedString::Chars & data = assert_cast<ColumnFixedString &>(column).getChars(); ColumnFixedString::Chars & data = assert_cast<ColumnFixedString &>(column).getChars();
size_t old_size = data.size(); size_t old_size = data.size();

View File

@ -15,10 +15,10 @@ public:
explicit SerializationFixedString(size_t n_) : n(n_) {} explicit SerializationFixedString(size_t n_) : n(n_) {}
size_t getN() const { return n; } size_t getN() const { return n; }
void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override; void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override;
void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override;

View File

@ -718,22 +718,22 @@ void SerializationLowCardinality::deserializeBinaryBulkWithMultipleStreams(
column = std::move(mutable_column); column = std::move(mutable_column);
} }
void SerializationLowCardinality::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationLowCardinality::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const
{ {
dictionary_type->getDefaultSerialization()->serializeBinary(field, ostr); dictionary_type->getDefaultSerialization()->serializeBinary(field, ostr, settings);
} }
void SerializationLowCardinality::deserializeBinary(Field & field, ReadBuffer & istr) const void SerializationLowCardinality::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const
{ {
dictionary_type->getDefaultSerialization()->deserializeBinary(field, istr); dictionary_type->getDefaultSerialization()->deserializeBinary(field, istr, settings);
} }
void SerializationLowCardinality::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const void SerializationLowCardinality::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{ {
serializeImpl(column, row_num, &ISerialization::serializeBinary, ostr); serializeImpl(column, row_num, &ISerialization::serializeBinary, ostr, settings);
} }
void SerializationLowCardinality::deserializeBinary(IColumn & column, ReadBuffer & istr) const void SerializationLowCardinality::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{ {
deserializeImpl(column, &ISerialization::deserializeBinary, istr); deserializeImpl(column, &ISerialization::deserializeBinary, istr, settings);
} }
void SerializationLowCardinality::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const void SerializationLowCardinality::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const

View File

@ -49,10 +49,10 @@ public:
DeserializeBinaryBulkStatePtr & state, DeserializeBinaryBulkStatePtr & state,
SubstreamsCache * cache) const override; SubstreamsCache * cache) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override; void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;

View File

@ -36,7 +36,7 @@ static IColumn & extractNestedColumn(IColumn & column)
return assert_cast<ColumnMap &>(column).getNestedColumn(); return assert_cast<ColumnMap &>(column).getNestedColumn();
} }
void SerializationMap::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationMap::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const
{ {
const auto & map = field.get<const Map &>(); const auto & map = field.get<const Map &>();
writeVarUInt(map.size(), ostr); writeVarUInt(map.size(), ostr);
@ -44,12 +44,12 @@ void SerializationMap::serializeBinary(const Field & field, WriteBuffer & ostr)
{ {
const auto & tuple = elem.safeGet<const Tuple>(); const auto & tuple = elem.safeGet<const Tuple>();
assert(tuple.size() == 2); assert(tuple.size() == 2);
key->serializeBinary(tuple[0], ostr); key->serializeBinary(tuple[0], ostr, settings);
value->serializeBinary(tuple[1], ostr); value->serializeBinary(tuple[1], ostr, settings);
} }
} }
void SerializationMap::deserializeBinary(Field & field, ReadBuffer & istr) const void SerializationMap::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const
{ {
size_t size; size_t size;
readVarUInt(size, istr); readVarUInt(size, istr);
@ -59,20 +59,20 @@ void SerializationMap::deserializeBinary(Field & field, ReadBuffer & istr) const
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
{ {
Tuple tuple(2); Tuple tuple(2);
key->deserializeBinary(tuple[0], istr); key->deserializeBinary(tuple[0], istr, settings);
value->deserializeBinary(tuple[1], istr); value->deserializeBinary(tuple[1], istr, settings);
map.push_back(std::move(tuple)); map.push_back(std::move(tuple));
} }
} }
void SerializationMap::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const void SerializationMap::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{ {
nested->serializeBinary(extractNestedColumn(column), row_num, ostr); nested->serializeBinary(extractNestedColumn(column), row_num, ostr, settings);
} }
void SerializationMap::deserializeBinary(IColumn & column, ReadBuffer & istr) const void SerializationMap::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{ {
nested->deserializeBinary(extractNestedColumn(column), istr); nested->deserializeBinary(extractNestedColumn(column), istr, settings);
} }

View File

@ -18,10 +18,10 @@ private:
public: public:
SerializationMap(const SerializationPtr & key_type_, const SerializationPtr & value_type_, const SerializationPtr & nested_); SerializationMap(const SerializationPtr & key_type_, const SerializationPtr & value_type_, const SerializationPtr & nested_);
void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override; void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override; void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override;
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;

View File

@ -19,10 +19,10 @@ private:
throw Exception("Serialization is not implemented for type Nothing", ErrorCodes::NOT_IMPLEMENTED); throw Exception("Serialization is not implemented for type Nothing", ErrorCodes::NOT_IMPLEMENTED);
} }
public: public:
void serializeBinary(const Field &, WriteBuffer &) const override { throwNoSerialization(); } void serializeBinary(const Field &, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void deserializeBinary(Field &, ReadBuffer &) const override { throwNoSerialization(); } void deserializeBinary(Field &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void serializeBinary(const IColumn &, size_t, WriteBuffer &) const override { throwNoSerialization(); } void serializeBinary(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void deserializeBinary(IColumn &, ReadBuffer &) const override { throwNoSerialization(); } void deserializeBinary(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void serializeText(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); } void serializeText(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void deserializeText(IColumn &, ReadBuffer &, const FormatSettings &, bool) const override { throwNoSerialization(); } void deserializeText(IColumn &, ReadBuffer &, const FormatSettings &, bool) const override { throwNoSerialization(); }

View File

@ -150,7 +150,7 @@ void SerializationNullable::deserializeBinaryBulkWithMultipleStreams(
} }
void SerializationNullable::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationNullable::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const
{ {
if (field.isNull()) if (field.isNull())
{ {
@ -159,17 +159,17 @@ void SerializationNullable::serializeBinary(const Field & field, WriteBuffer & o
else else
{ {
writeBinary(false, ostr); writeBinary(false, ostr);
nested->serializeBinary(field, ostr); nested->serializeBinary(field, ostr, settings);
} }
} }
void SerializationNullable::deserializeBinary(Field & field, ReadBuffer & istr) const void SerializationNullable::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const
{ {
bool is_null = false; bool is_null = false;
readBinary(is_null, istr); readBinary(is_null, istr);
if (!is_null) if (!is_null)
{ {
nested->deserializeBinary(field, istr); nested->deserializeBinary(field, istr, settings);
} }
else else
{ {
@ -177,14 +177,14 @@ void SerializationNullable::deserializeBinary(Field & field, ReadBuffer & istr)
} }
} }
void SerializationNullable::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const void SerializationNullable::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{ {
const ColumnNullable & col = assert_cast<const ColumnNullable &>(column); const ColumnNullable & col = assert_cast<const ColumnNullable &>(column);
bool is_null = col.isNullAt(row_num); bool is_null = col.isNullAt(row_num);
writeBinary(is_null, ostr); writeBinary(is_null, ostr);
if (!is_null) if (!is_null)
nested->serializeBinary(col.getNestedColumn(), row_num, ostr); nested->serializeBinary(col.getNestedColumn(), row_num, ostr, settings);
} }
/// Deserialize value into ColumnNullable. /// Deserialize value into ColumnNullable.
@ -235,11 +235,11 @@ static ReturnType safeDeserialize(
} }
void SerializationNullable::deserializeBinary(IColumn & column, ReadBuffer & istr) const void SerializationNullable::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{ {
safeDeserialize(column, *nested, safeDeserialize(column, *nested,
[&istr] { bool is_null = false; readBinary(is_null, istr); return is_null; }, [&istr] { bool is_null = false; readBinary(is_null, istr); return is_null; },
[this, &istr] (IColumn & nested_column) { nested->deserializeBinary(nested_column, istr); }); [this, &istr, settings] (IColumn & nested_column) { nested->deserializeBinary(nested_column, istr, settings); });
} }

View File

@ -45,10 +45,10 @@ public:
DeserializeBinaryBulkStatePtr & state, DeserializeBinaryBulkStatePtr & state,
SubstreamsCache * cache) const override; SubstreamsCache * cache) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override; void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;

View File

@ -102,7 +102,7 @@ void SerializationNumber<T>::deserializeTextCSV(IColumn & column, ReadBuffer & i
} }
template <typename T> template <typename T>
void SerializationNumber<T>::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationNumber<T>::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const
{ {
/// ColumnVector<T>::ValueType is a narrower type. For example, UInt8, when the Field type is UInt64 /// ColumnVector<T>::ValueType is a narrower type. For example, UInt8, when the Field type is UInt64
typename ColumnVector<T>::ValueType x = static_cast<typename ColumnVector<T>::ValueType>(field.get<FieldType>()); typename ColumnVector<T>::ValueType x = static_cast<typename ColumnVector<T>::ValueType>(field.get<FieldType>());
@ -110,7 +110,7 @@ void SerializationNumber<T>::serializeBinary(const Field & field, WriteBuffer &
} }
template <typename T> template <typename T>
void SerializationNumber<T>::deserializeBinary(Field & field, ReadBuffer & istr) const void SerializationNumber<T>::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const
{ {
typename ColumnVector<T>::ValueType x; typename ColumnVector<T>::ValueType x;
readBinary(x, istr); readBinary(x, istr);
@ -118,13 +118,13 @@ void SerializationNumber<T>::deserializeBinary(Field & field, ReadBuffer & istr)
} }
template <typename T> template <typename T>
void SerializationNumber<T>::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const void SerializationNumber<T>::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
{ {
writeBinary(assert_cast<const ColumnVector<T> &>(column).getData()[row_num], ostr); writeBinary(assert_cast<const ColumnVector<T> &>(column).getData()[row_num], ostr);
} }
template <typename T> template <typename T>
void SerializationNumber<T>::deserializeBinary(IColumn & column, ReadBuffer & istr) const void SerializationNumber<T>::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{ {
typename ColumnVector<T>::ValueType x; typename ColumnVector<T>::ValueType x;
readBinary(x, istr); readBinary(x, istr);

View File

@ -22,10 +22,10 @@ public:
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
/** Format is platform-dependent. */ /** Format is platform-dependent. */
void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override; void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override;
void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override;
}; };

View File

@ -376,25 +376,25 @@ void SerializationObject<Parser>::deserializeBinaryBulkFromTuple(
} }
template <typename Parser> template <typename Parser>
void SerializationObject<Parser>::serializeBinary(const Field &, WriteBuffer &) const void SerializationObject<Parser>::serializeBinary(const Field &, WriteBuffer &, const FormatSettings &) const
{ {
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for SerializationObject"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for SerializationObject");
} }
template <typename Parser> template <typename Parser>
void SerializationObject<Parser>::deserializeBinary(Field &, ReadBuffer &) const void SerializationObject<Parser>::deserializeBinary(Field &, ReadBuffer &, const FormatSettings &) const
{ {
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for SerializationObject"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for SerializationObject");
} }
template <typename Parser> template <typename Parser>
void SerializationObject<Parser>::serializeBinary(const IColumn &, size_t, WriteBuffer &) const void SerializationObject<Parser>::serializeBinary(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const
{ {
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for SerializationObject"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for SerializationObject");
} }
template <typename Parser> template <typename Parser>
void SerializationObject<Parser>::deserializeBinary(IColumn &, ReadBuffer &) const void SerializationObject<Parser>::deserializeBinary(IColumn &, ReadBuffer &, const FormatSettings &) const
{ {
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for SerializationObject"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for SerializationObject");
} }

View File

@ -57,10 +57,10 @@ public:
DeserializeBinaryBulkStatePtr & state, DeserializeBinaryBulkStatePtr & state,
SubstreamsCache * cache) const override; SubstreamsCache * cache) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override; void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;

View File

@ -302,23 +302,23 @@ void SerializationSparse::deserializeBinaryBulkWithMultipleStreams(
/// All methods below just wrap nested serialization. /// All methods below just wrap nested serialization.
void SerializationSparse::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationSparse::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const
{ {
nested->serializeBinary(field, ostr); nested->serializeBinary(field, ostr, settings);
} }
void SerializationSparse::deserializeBinary(Field & field, ReadBuffer & istr) const void SerializationSparse::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const
{ {
nested->deserializeBinary(field, istr); nested->deserializeBinary(field, istr, settings);
} }
void SerializationSparse::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const void SerializationSparse::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{ {
const auto & column_sparse = assert_cast<const ColumnSparse &>(column); const auto & column_sparse = assert_cast<const ColumnSparse &>(column);
nested->serializeBinary(column_sparse.getValuesColumn(), column_sparse.getValueIndex(row_num), ostr); nested->serializeBinary(column_sparse.getValuesColumn(), column_sparse.getValueIndex(row_num), ostr, settings);
} }
void SerializationSparse::deserializeBinary(IColumn &, ReadBuffer &) const void SerializationSparse::deserializeBinary(IColumn &, ReadBuffer &, const FormatSettings &) const
{ {
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'deserializeBinary' is not implemented for SerializationSparse"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'deserializeBinary' is not implemented for SerializationSparse");
} }

View File

@ -61,11 +61,11 @@ public:
DeserializeBinaryBulkStatePtr & state, DeserializeBinaryBulkStatePtr & state,
SubstreamsCache * cache) const override; SubstreamsCache * cache) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override; void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;

View File

@ -25,20 +25,37 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int INCORRECT_DATA; extern const int INCORRECT_DATA;
extern const int TOO_LARGE_STRING_SIZE;
} }
void SerializationString::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationString::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const
{ {
const String & s = field.get<const String &>(); const String & s = field.get<const String &>();
if (settings.max_binary_string_size && s.size() > settings.max_binary_string_size)
throw Exception(
ErrorCodes::TOO_LARGE_STRING_SIZE,
"Too large string size: {}. The maximum is: {}. To increase the maximum, use setting "
"format_binary_max_string_size",
s.size(),
settings.max_binary_string_size);
writeVarUInt(s.size(), ostr); writeVarUInt(s.size(), ostr);
writeString(s, ostr); writeString(s, ostr);
} }
void SerializationString::deserializeBinary(Field & field, ReadBuffer & istr) const void SerializationString::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const
{ {
UInt64 size; UInt64 size;
readVarUInt(size, istr); readVarUInt(size, istr);
if (settings.max_binary_string_size && size > settings.max_binary_string_size)
throw Exception(
ErrorCodes::TOO_LARGE_STRING_SIZE,
"Too large string size: {}. The maximum is: {}. To increase the maximum, use setting "
"format_binary_max_string_size",
size,
settings.max_binary_string_size);
field = String(); field = String();
String & s = field.get<String &>(); String & s = field.get<String &>();
s.resize(size); s.resize(size);
@ -46,15 +63,23 @@ void SerializationString::deserializeBinary(Field & field, ReadBuffer & istr) co
} }
void SerializationString::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const void SerializationString::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{ {
const StringRef & s = assert_cast<const ColumnString &>(column).getDataAt(row_num); const StringRef & s = assert_cast<const ColumnString &>(column).getDataAt(row_num);
if (settings.max_binary_string_size && s.size > settings.max_binary_string_size)
throw Exception(
ErrorCodes::TOO_LARGE_STRING_SIZE,
"Too large string size: {}. The maximum is: {}. To increase the maximum, use setting "
"format_binary_max_string_size",
s.size,
settings.max_binary_string_size);
writeVarUInt(s.size, ostr); writeVarUInt(s.size, ostr);
writeString(s, ostr); writeString(s, ostr);
} }
void SerializationString::deserializeBinary(IColumn & column, ReadBuffer & istr) const void SerializationString::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{ {
ColumnString & column_string = assert_cast<ColumnString &>(column); ColumnString & column_string = assert_cast<ColumnString &>(column);
ColumnString::Chars & data = column_string.getChars(); ColumnString::Chars & data = column_string.getChars();
@ -62,6 +87,13 @@ void SerializationString::deserializeBinary(IColumn & column, ReadBuffer & istr)
UInt64 size; UInt64 size;
readVarUInt(size, istr); readVarUInt(size, istr);
if (settings.max_binary_string_size && size > settings.max_binary_string_size)
throw Exception(
ErrorCodes::TOO_LARGE_STRING_SIZE,
"Too large string size: {}. The maximum is: {}. To increase the maximum, use setting "
"format_binary_max_string_size",
size,
settings.max_binary_string_size);
size_t old_chars_size = data.size(); size_t old_chars_size = data.size();
size_t offset = old_chars_size + size + 1; size_t offset = old_chars_size + size + 1;

View File

@ -8,10 +8,10 @@ namespace DB
class SerializationString final : public ISerialization class SerializationString final : public ISerialization
{ {
public: public:
void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override; void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override;
void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override;

View File

@ -29,17 +29,17 @@ static inline const IColumn & extractElementColumn(const IColumn & column, size_
return assert_cast<const ColumnTuple &>(column).getColumn(idx); return assert_cast<const ColumnTuple &>(column).getColumn(idx);
} }
void SerializationTuple::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationTuple::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const
{ {
const auto & tuple = field.get<const Tuple &>(); const auto & tuple = field.get<const Tuple &>();
for (size_t element_index = 0; element_index < elems.size(); ++element_index) for (size_t element_index = 0; element_index < elems.size(); ++element_index)
{ {
const auto & serialization = elems[element_index]; const auto & serialization = elems[element_index];
serialization->serializeBinary(tuple[element_index], ostr); serialization->serializeBinary(tuple[element_index], ostr, settings);
} }
} }
void SerializationTuple::deserializeBinary(Field & field, ReadBuffer & istr) const void SerializationTuple::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const
{ {
const size_t size = elems.size(); const size_t size = elems.size();
@ -47,15 +47,15 @@ void SerializationTuple::deserializeBinary(Field & field, ReadBuffer & istr) con
Tuple & tuple = field.get<Tuple &>(); Tuple & tuple = field.get<Tuple &>();
tuple.reserve(size); tuple.reserve(size);
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
elems[i]->deserializeBinary(tuple.emplace_back(), istr); elems[i]->deserializeBinary(tuple.emplace_back(), istr, settings);
} }
void SerializationTuple::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const void SerializationTuple::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{ {
for (size_t element_index = 0; element_index < elems.size(); ++element_index) for (size_t element_index = 0; element_index < elems.size(); ++element_index)
{ {
const auto & serialization = elems[element_index]; const auto & serialization = elems[element_index];
serialization->serializeBinary(extractElementColumn(column, element_index), row_num, ostr); serialization->serializeBinary(extractElementColumn(column, element_index), row_num, ostr, settings);
} }
} }
@ -97,12 +97,12 @@ static void addElementSafe(size_t num_elems, IColumn & column, F && impl)
} }
} }
void SerializationTuple::deserializeBinary(IColumn & column, ReadBuffer & istr) const void SerializationTuple::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{ {
addElementSafe(elems.size(), column, [&] addElementSafe(elems.size(), column, [&]
{ {
for (size_t i = 0; i < elems.size(); ++i) for (size_t i = 0; i < elems.size(); ++i)
elems[i]->deserializeBinary(extractElementColumn(column, i), istr); elems[i]->deserializeBinary(extractElementColumn(column, i), istr, settings);
}); });
} }

View File

@ -17,10 +17,10 @@ public:
{ {
} }
void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override; void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override; void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override;
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;

View File

@ -82,25 +82,25 @@ void SerializationUUID::deserializeTextCSV(IColumn & column, ReadBuffer & istr,
} }
void SerializationUUID::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationUUID::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const
{ {
UUID x = field.get<UUID>(); UUID x = field.get<UUID>();
writeBinary(x, ostr); writeBinary(x, ostr);
} }
void SerializationUUID::deserializeBinary(Field & field, ReadBuffer & istr) const void SerializationUUID::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const
{ {
UUID x; UUID x;
readBinary(x, istr); readBinary(x, istr);
field = NearestFieldType<UUID>(x); field = NearestFieldType<UUID>(x);
} }
void SerializationUUID::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const void SerializationUUID::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
{ {
writeBinary(assert_cast<const ColumnVector<UUID> &>(column).getData()[row_num], ostr); writeBinary(assert_cast<const ColumnVector<UUID> &>(column).getData()[row_num], ostr);
} }
void SerializationUUID::deserializeBinary(IColumn & column, ReadBuffer & istr) const void SerializationUUID::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{ {
UUID x; UUID x;
readBinary(x, istr); readBinary(x, istr);

View File

@ -19,10 +19,10 @@ public:
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override; void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override;
void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override;
}; };

View File

@ -66,24 +66,24 @@ void SerializationWrapper::deserializeBinaryBulk(IColumn & column, ReadBuffer &
nested_serialization->deserializeBinaryBulk(column, istr, limit, avg_value_size_hint); nested_serialization->deserializeBinaryBulk(column, istr, limit, avg_value_size_hint);
} }
void SerializationWrapper::serializeBinary(const Field & field, WriteBuffer & ostr) const void SerializationWrapper::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const
{ {
nested_serialization->serializeBinary(field, ostr); nested_serialization->serializeBinary(field, ostr, settings);
} }
void SerializationWrapper::deserializeBinary(Field & field, ReadBuffer & istr) const void SerializationWrapper::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const
{ {
nested_serialization->deserializeBinary(field, istr); nested_serialization->deserializeBinary(field, istr, settings);
} }
void SerializationWrapper::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const void SerializationWrapper::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{ {
nested_serialization->serializeBinary(column, row_num, ostr); nested_serialization->serializeBinary(column, row_num, ostr, settings);
} }
void SerializationWrapper::deserializeBinary(IColumn & column, ReadBuffer & istr) const void SerializationWrapper::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{ {
nested_serialization->deserializeBinary(column, istr); nested_serialization->deserializeBinary(column, istr, settings);
} }
void SerializationWrapper::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const void SerializationWrapper::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const

View File

@ -55,11 +55,11 @@ public:
void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override;
void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override; void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;

View File

@ -1,109 +1,73 @@
#include <Databases/DDLDependencyVisitor.h> #include <Databases/DDLDependencyVisitor.h>
#include <Dictionaries/getDictionaryConfigurationFromAST.h> #include <Dictionaries/getDictionaryConfigurationFromAST.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTCreateQuery.h> #include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSelectWithUnionQuery.h> #include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Poco/String.h> #include <Poco/String.h>
namespace DB namespace DB
{ {
using TableLoadingDependenciesVisitor = DDLDependencyVisitor::Visitor; namespace
TableNamesSet getDependenciesSetFromCreateQuery(ContextPtr global_context, const QualifiedTableName & table, const ASTPtr & ast)
{ {
assert(global_context == global_context->getGlobalContext()); /// CREATE TABLE or CREATE DICTIONARY or CREATE VIEW or CREATE TEMPORARY TABLE or CREATE DATABASE query.
TableLoadingDependenciesVisitor::Data data; void visitCreateQuery(const ASTCreateQuery & create, DDLDependencyVisitor::Data & data)
data.default_database = global_context->getCurrentDatabase(); {
data.create_query = ast; QualifiedTableName to_table{create.to_table_id.database_name, create.to_table_id.table_name};
data.global_context = global_context; if (!to_table.table.empty())
TableLoadingDependenciesVisitor visitor{data}; {
visitor.visit(ast); /// TO target_table (for materialized views)
data.dependencies.erase(table); if (to_table.database.empty())
return data.dependencies; to_table.database = data.default_database;
data.dependencies.emplace(to_table);
} }
void DDLDependencyVisitor::visit(const ASTPtr & ast, Data & data) QualifiedTableName as_table{create.as_database, create.as_table};
if (!as_table.table.empty())
{ {
/// Looking for functions in column default expressions and dictionary source definition /// AS table_name
if (const auto * function = ast->as<ASTFunction>()) if (as_table.database.empty())
visit(*function, data); as_table.database = data.default_database;
else if (const auto * dict_source = ast->as<ASTFunctionWithKeyValueArguments>()) data.dependencies.emplace(as_table);
visit(*dict_source, data); }
else if (const auto * storage = ast->as<ASTStorage>())
visit(*storage, data);
} }
bool DDLMatcherBase::needChildVisit(const ASTPtr & node, const ASTPtr & child) /// ASTTableExpression represents a reference to a table in SELECT query.
/// DDLDependencyVisitor should handle ASTTableExpression because some CREATE queries can contain SELECT queries after AS
/// (for example, CREATE VIEW).
void visitTableExpression(const ASTTableExpression & expr, DDLDependencyVisitor::Data & data)
{ {
if (node->as<ASTStorage>()) if (!expr.database_and_table_name)
return false;
if (auto * create = node->as<ASTCreateQuery>())
{
if (child.get() == create->select)
return false;
}
return true;
}
ssize_t DDLMatcherBase::getPositionOfTableNameArgument(const ASTFunction & function)
{
if (function.name == "joinGet" ||
function.name == "dictHas" ||
function.name == "dictIsIn" ||
function.name.starts_with("dictGet"))
return 0;
if (Poco::toLower(function.name) == "in")
return 1;
return -1;
}
void DDLDependencyVisitor::visit(const ASTFunction & function, Data & data)
{
ssize_t table_name_arg_idx = getPositionOfTableNameArgument(function);
if (table_name_arg_idx < 0)
return;
extractTableNameFromArgument(function, data, table_name_arg_idx);
}
void DDLDependencyVisitor::visit(const ASTFunctionWithKeyValueArguments & dict_source, Data & data)
{
if (dict_source.name != "clickhouse")
return;
if (!dict_source.elements)
return; return;
auto config = getDictionaryConfigurationFromAST(data.create_query->as<ASTCreateQuery &>(), data.global_context); const ASTIdentifier * identifier = dynamic_cast<const ASTIdentifier *>(expr.database_and_table_name.get());
auto info = getInfoIfClickHouseDictionarySource(config, data.global_context); if (!identifier)
if (!info || !info->is_local)
return; return;
if (info->table_name.database.empty()) auto table_identifier = identifier->createTable();
info->table_name.database = data.default_database; if (!table_identifier)
data.dependencies.emplace(std::move(info->table_name)); return;
}
void DDLDependencyVisitor::visit(const ASTStorage & storage, Data & data) QualifiedTableName qualified_name{table_identifier->getDatabaseName(), table_identifier->shortName()};
if (qualified_name.table.empty())
return;
if (qualified_name.database.empty())
{ {
if (!storage.engine) /// It can be table/dictionary from default database or XML dictionary, but we cannot distinguish it here.
return; qualified_name.database = data.default_database;
if (storage.engine->name != "Dictionary")
return;
extractTableNameFromArgument(*storage.engine, data, 0);
} }
data.dependencies.emplace(qualified_name);
}
void DDLDependencyVisitor::extractTableNameFromArgument(const ASTFunction & function, Data & data, size_t arg_idx) /// Extracts a table name with optional database written in the form db_name.table_name (as identifier) or 'db_name.table_name' (as string).
void extractQualifiedTableNameFromArgument(const ASTFunction & function, DDLDependencyVisitor::Data & data, size_t arg_idx)
{ {
/// Just ignore incorrect arguments, proper exception will be thrown later /// Just ignore incorrect arguments, proper exception will be thrown later
if (!function.arguments || function.arguments->children.size() <= arg_idx) if (!function.arguments || function.arguments->children.size() <= arg_idx)
@ -111,7 +75,11 @@ void DDLDependencyVisitor::extractTableNameFromArgument(const ASTFunction & func
QualifiedTableName qualified_name; QualifiedTableName qualified_name;
const auto * arg = function.arguments->as<ASTExpressionList>()->children[arg_idx].get(); const auto * expr_list = function.arguments->as<ASTExpressionList>();
if (!expr_list)
return;
const auto * arg = expr_list->children[arg_idx].get();
if (const auto * literal = arg->as<ASTLiteral>()) if (const auto * literal = arg->as<ASTLiteral>())
{ {
if (literal->value.getType() != Field::Types::String) if (literal->value.getType() != Field::Types::String)
@ -137,7 +105,7 @@ void DDLDependencyVisitor::extractTableNameFromArgument(const ASTFunction & func
} }
else else
{ {
assert(false); /// Just return because we don't validate AST in this function.
return; return;
} }
@ -149,50 +117,123 @@ void DDLDependencyVisitor::extractTableNameFromArgument(const ASTFunction & func
data.dependencies.emplace(std::move(qualified_name)); data.dependencies.emplace(std::move(qualified_name));
} }
/// Extracts a table name with database written in the form 'db_name', 'table_name' (two strings).
void NormalizeAndEvaluateConstants::visit(const ASTPtr & ast, Data & data) void extractDatabaseAndTableNameFromArguments(const ASTFunction & function, DDLDependencyVisitor::Data & data, size_t database_arg_idx, size_t table_arg_idx)
{ {
assert(data.create_query_context->hasQueryContext()); /// Just ignore incorrect arguments, proper exception will be thrown later
if (!function.arguments || (function.arguments->children.size() <= database_arg_idx)
/// Looking for functions in column default expressions and dictionary source definition || (function.arguments->children.size() <= table_arg_idx))
if (const auto * function = ast->as<ASTFunction>())
visit(*function, data);
else if (const auto * dict_source = ast->as<ASTFunctionWithKeyValueArguments>())
visit(*dict_source, data);
}
void NormalizeAndEvaluateConstants::visit(const ASTFunction & function, Data & data)
{
/// Replace expressions like "dictGet(currentDatabase() || '.dict', 'value', toUInt32(1))"
/// with "dictGet('db_name.dict', 'value', toUInt32(1))"
ssize_t table_name_arg_idx = getPositionOfTableNameArgument(function);
if (table_name_arg_idx < 0)
return; return;
if (!function.arguments || function.arguments->children.size() <= static_cast<size_t>(table_name_arg_idx)) const auto * expr_list = function.arguments->as<ASTExpressionList>();
if (!expr_list)
return; return;
auto & arg = function.arguments->as<ASTExpressionList &>().children[table_name_arg_idx]; const auto * database_literal = expr_list->children[database_arg_idx]->as<ASTLiteral>();
if (arg->as<ASTFunction>()) const auto * table_name_literal = expr_list->children[table_arg_idx]->as<ASTLiteral>();
arg = evaluateConstantExpressionAsLiteral(arg, data.create_query_context);
}
if (!database_literal || !table_name_literal || (database_literal->value.getType() != Field::Types::String)
void NormalizeAndEvaluateConstants::visit(const ASTFunctionWithKeyValueArguments & dict_source, Data & data) || (table_name_literal->value.getType() != Field::Types::String))
{
if (!dict_source.elements)
return; return;
auto & expr_list = dict_source.elements->as<ASTExpressionList &>(); QualifiedTableName qualified_name{database_literal->value.get<String>(), table_name_literal->value.get<String>()};
for (auto & child : expr_list.children) if (qualified_name.table.empty())
{ return;
ASTPair * pair = child->as<ASTPair>();
if (pair->second->as<ASTFunction>()) if (qualified_name.database.empty())
{ qualified_name.database = data.default_database;
auto ast_literal = evaluateConstantExpressionAsLiteral(pair->children[0], data.create_query_context);
pair->replace(pair->second, ast_literal); data.dependencies.emplace(qualified_name);
} }
void visitFunction(const ASTFunction & function, DDLDependencyVisitor::Data & data)
{
if (function.name == "joinGet" || function.name == "dictHas" || function.name == "dictIsIn" || function.name.starts_with("dictGet"))
{
/// dictGet('dict_name', attr_names, id_expr)
/// dictHas('dict_name', id_expr)
/// joinGet(join_storage_table_name, `value_column`, join_keys)
extractQualifiedTableNameFromArgument(function, data, 0);
}
else if (function.name == "in" || function.name == "notIn" || function.name == "globalIn" || function.name == "globalNotIn")
{
/// in(x, table_name) - function for evaluating (x IN table_name)
extractQualifiedTableNameFromArgument(function, data, 1);
}
else if (function.name == "dictionary")
{
/// dictionary(dict_name)
extractQualifiedTableNameFromArgument(function, data, 0);
} }
} }
void visitTableEngine(const ASTFunction & table_engine, DDLDependencyVisitor::Data & data)
{
if (table_engine.name == "Dictionary")
extractQualifiedTableNameFromArgument(table_engine, data, 0);
if (table_engine.name == "Buffer")
extractDatabaseAndTableNameFromArguments(table_engine, data, 0, 1);
}
void visitDictionaryDef(const ASTDictionary & dictionary, DDLDependencyVisitor::Data & data)
{
if (!dictionary.source || dictionary.source->name != "clickhouse" || !dictionary.source->elements)
return;
auto config = getDictionaryConfigurationFromAST(data.create_query->as<ASTCreateQuery &>(), data.global_context);
auto info = getInfoIfClickHouseDictionarySource(config, data.global_context);
if (!info || !info->is_local)
return;
if (info->table_name.database.empty())
info->table_name.database = data.default_database;
data.dependencies.emplace(std::move(info->table_name));
}
}
TableNamesSet getDependenciesFromCreateQuery(const ContextPtr & global_context, const QualifiedTableName & table_name, const ASTPtr & ast)
{
assert(global_context == global_context->getGlobalContext());
DDLDependencyVisitor::Data data;
data.table_name = table_name;
data.default_database = global_context->getCurrentDatabase();
data.create_query = ast;
data.global_context = global_context;
DDLDependencyVisitor::Visitor visitor{data};
visitor.visit(ast);
data.dependencies.erase(data.table_name);
return data.dependencies;
}
void DDLDependencyVisitor::visit(const ASTPtr & ast, Data & data)
{
if (auto * create = ast->as<ASTCreateQuery>())
{
visitCreateQuery(*create, data);
}
else if (auto * dictionary = ast->as<ASTDictionary>())
{
visitDictionaryDef(*dictionary, data);
}
else if (auto * expr = ast->as<ASTTableExpression>())
{
visitTableExpression(*expr, data);
}
else if (const auto * function = ast->as<ASTFunction>())
{
if (function->kind == ASTFunction::Kind::TABLE_ENGINE)
visitTableEngine(*function, data);
else
visitFunction(*function, data);
}
}
bool DDLDependencyVisitor::needChildVisit(const ASTPtr &, const ASTPtr &)
{
return true;
}
} }

View File

@ -1,72 +1,36 @@
#pragma once #pragma once
#include <Core/QualifiedTableName.h>
#include <Parsers/IAST_fwd.h> #include <Parsers/IAST_fwd.h>
#include <Interpreters/InDepthNodeVisitor.h> #include <Interpreters/InDepthNodeVisitor.h>
#include <Core/QualifiedTableName.h>
namespace DB namespace DB
{ {
class ASTFunction;
class ASTFunctionWithKeyValueArguments;
class ASTStorage;
using TableNamesSet = std::unordered_set<QualifiedTableName>; using TableNamesSet = std::unordered_set<QualifiedTableName>;
TableNamesSet getDependenciesSetFromCreateQuery(ContextPtr global_context, const QualifiedTableName & table, const ASTPtr & ast); /// Returns a list of all tables explicitly referenced in the create query of a specified table.
/// For example, a column default expression can use dictGet() and thus reference a dictionary.
class DDLMatcherBase
{
public:
static bool needChildVisit(const ASTPtr & node, const ASTPtr & child);
static ssize_t getPositionOfTableNameArgument(const ASTFunction & function);
};
/// Visits ASTCreateQuery and extracts names of table (or dictionary) dependencies
/// from column default expressions (joinGet, dictGet, etc)
/// or dictionary source (for dictionaries from local ClickHouse table).
/// Does not validate AST, works a best-effort way. /// Does not validate AST, works a best-effort way.
class DDLDependencyVisitor : public DDLMatcherBase TableNamesSet getDependenciesFromCreateQuery(const ContextPtr & global_context, const QualifiedTableName & table_name, const ASTPtr & ast);
/// Visits ASTCreateQuery and extracts the names of all tables explicitly referenced in the create query.
class DDLDependencyVisitor
{ {
public: public:
struct Data struct Data
{ {
String default_database;
TableNamesSet dependencies;
ContextPtr global_context;
ASTPtr create_query; ASTPtr create_query;
QualifiedTableName table_name;
String default_database;
ContextPtr global_context;
TableNamesSet dependencies;
}; };
using Visitor = ConstInDepthNodeVisitor<DDLDependencyVisitor, true>; using Visitor = ConstInDepthNodeVisitor<DDLDependencyVisitor, /* top_to_bottom= */ true>;
static void visit(const ASTPtr & ast, Data & data); static void visit(const ASTPtr & ast, Data & data);
static bool needChildVisit(const ASTPtr & node, const ASTPtr & child);
private:
static void visit(const ASTFunction & function, Data & data);
static void visit(const ASTFunctionWithKeyValueArguments & dict_source, Data & data);
static void visit(const ASTStorage & storage, Data & data);
static void extractTableNameFromArgument(const ASTFunction & function, Data & data, size_t arg_idx);
}; };
class NormalizeAndEvaluateConstants : public DDLMatcherBase
{
public:
struct Data
{
ContextPtr create_query_context;
};
using Visitor = ConstInDepthNodeVisitor<NormalizeAndEvaluateConstants, true>;
static void visit(const ASTPtr & ast, Data & data);
private:
static void visit(const ASTFunction & function, Data & data);
static void visit(const ASTFunctionWithKeyValueArguments & dict_source, Data & data);
};
using NormalizeAndEvaluateConstantsVisitor = NormalizeAndEvaluateConstants::Visitor;
} }

View File

@ -0,0 +1,152 @@
#include <Databases/DDLLoadingDependencyVisitor.h>
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Poco/String.h>
namespace DB
{
using TableLoadingDependenciesVisitor = DDLLoadingDependencyVisitor::Visitor;
TableNamesSet getLoadingDependenciesFromCreateQuery(ContextPtr global_context, const QualifiedTableName & table, const ASTPtr & ast)
{
assert(global_context == global_context->getGlobalContext());
TableLoadingDependenciesVisitor::Data data;
data.default_database = global_context->getCurrentDatabase();
data.create_query = ast;
data.global_context = global_context;
TableLoadingDependenciesVisitor visitor{data};
visitor.visit(ast);
data.dependencies.erase(table);
return data.dependencies;
}
void DDLLoadingDependencyVisitor::visit(const ASTPtr & ast, Data & data)
{
/// Looking for functions in column default expressions and dictionary source definition
if (const auto * function = ast->as<ASTFunction>())
visit(*function, data);
else if (const auto * dict_source = ast->as<ASTFunctionWithKeyValueArguments>())
visit(*dict_source, data);
else if (const auto * storage = ast->as<ASTStorage>())
visit(*storage, data);
}
bool DDLMatcherBase::needChildVisit(const ASTPtr & node, const ASTPtr & child)
{
if (node->as<ASTStorage>())
return false;
if (auto * create = node->as<ASTCreateQuery>())
{
if (child.get() == create->select)
return false;
}
return true;
}
ssize_t DDLMatcherBase::getPositionOfTableNameArgument(const ASTFunction & function)
{
if (function.name == "joinGet" ||
function.name == "dictHas" ||
function.name == "dictIsIn" ||
function.name.starts_with("dictGet"))
return 0;
if (Poco::toLower(function.name) == "in")
return 1;
return -1;
}
void DDLLoadingDependencyVisitor::visit(const ASTFunction & function, Data & data)
{
ssize_t table_name_arg_idx = getPositionOfTableNameArgument(function);
if (table_name_arg_idx < 0)
return;
extractTableNameFromArgument(function, data, table_name_arg_idx);
}
void DDLLoadingDependencyVisitor::visit(const ASTFunctionWithKeyValueArguments & dict_source, Data & data)
{
if (dict_source.name != "clickhouse")
return;
if (!dict_source.elements)
return;
auto config = getDictionaryConfigurationFromAST(data.create_query->as<ASTCreateQuery &>(), data.global_context);
auto info = getInfoIfClickHouseDictionarySource(config, data.global_context);
if (!info || !info->is_local)
return;
if (info->table_name.database.empty())
info->table_name.database = data.default_database;
data.dependencies.emplace(std::move(info->table_name));
}
void DDLLoadingDependencyVisitor::visit(const ASTStorage & storage, Data & data)
{
if (!storage.engine)
return;
if (storage.engine->name != "Dictionary")
return;
extractTableNameFromArgument(*storage.engine, data, 0);
}
void DDLLoadingDependencyVisitor::extractTableNameFromArgument(const ASTFunction & function, Data & data, size_t arg_idx)
{
/// Just ignore incorrect arguments, proper exception will be thrown later
if (!function.arguments || function.arguments->children.size() <= arg_idx)
return;
QualifiedTableName qualified_name;
const auto * arg = function.arguments->as<ASTExpressionList>()->children[arg_idx].get();
if (const auto * literal = arg->as<ASTLiteral>())
{
if (literal->value.getType() != Field::Types::String)
return;
auto maybe_qualified_name = QualifiedTableName::tryParseFromString(literal->value.get<String>());
/// Just return if name if invalid
if (!maybe_qualified_name)
return;
qualified_name = std::move(*maybe_qualified_name);
}
else if (const auto * identifier = dynamic_cast<const ASTIdentifier *>(arg))
{
/// ASTIdentifier or ASTTableIdentifier
auto table_identifier = identifier->createTable();
/// Just return if table identified is invalid
if (!table_identifier)
return;
qualified_name.database = table_identifier->getDatabaseName();
qualified_name.table = table_identifier->shortName();
}
else
{
assert(false);
return;
}
if (qualified_name.database.empty())
{
/// It can be table/dictionary from default database or XML dictionary, but we cannot distinguish it here.
qualified_name.database = data.default_database;
}
data.dependencies.emplace(std::move(qualified_name));
}
}

View File

@ -0,0 +1,54 @@
#pragma once
#include <Core/QualifiedTableName.h>
#include <Parsers/IAST_fwd.h>
#include <Interpreters/InDepthNodeVisitor.h>
namespace DB
{
class ASTFunction;
class ASTFunctionWithKeyValueArguments;
class ASTStorage;
using TableNamesSet = std::unordered_set<QualifiedTableName>;
/// Returns a list of all tables which should be loaded before a specified table.
/// For example, a local ClickHouse table should be loaded before a dictionary which uses that table as its source.
/// Does not validate AST, works a best-effort way.
TableNamesSet getLoadingDependenciesFromCreateQuery(ContextPtr global_context, const QualifiedTableName & table, const ASTPtr & ast);
class DDLMatcherBase
{
public:
static bool needChildVisit(const ASTPtr & node, const ASTPtr & child);
static ssize_t getPositionOfTableNameArgument(const ASTFunction & function);
};
/// Visits ASTCreateQuery and extracts the names of all tables which should be loaded before a specified table.
/// TODO: Combine this class with DDLDependencyVisitor (because loading dependencies are a subset of referential dependencies).
class DDLLoadingDependencyVisitor : public DDLMatcherBase
{
public:
struct Data
{
String default_database;
TableNamesSet dependencies;
ContextPtr global_context;
ASTPtr create_query;
};
using Visitor = ConstInDepthNodeVisitor<DDLLoadingDependencyVisitor, true>;
static void visit(const ASTPtr & ast, Data & data);
private:
static void visit(const ASTFunction & function, Data & data);
static void visit(const ASTFunctionWithKeyValueArguments & dict_source, Data & data);
static void visit(const ASTStorage & storage, Data & data);
static void extractTableNameFromArgument(const ASTFunction & function, Data & data, size_t arg_idx);
};
}

View File

@ -2,7 +2,7 @@
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
#include <Databases/DatabaseMemory.h> #include <Databases/DatabaseMemory.h>
#include <Databases/DatabasesCommon.h> #include <Databases/DatabasesCommon.h>
#include <Databases/DDLDependencyVisitor.h> #include <Databases/DDLLoadingDependencyVisitor.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Parsers/ASTCreateQuery.h> #include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
@ -142,8 +142,9 @@ void DatabaseMemory::alterTable(ContextPtr local_context, const StorageID & tabl
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Cannot alter: There is no metadata of table {}", table_id.getNameForLogs()); throw Exception(ErrorCodes::UNKNOWN_TABLE, "Cannot alter: There is no metadata of table {}", table_id.getNameForLogs());
applyMetadataChangesToCreateQuery(it->second, metadata); applyMetadataChangesToCreateQuery(it->second, metadata);
TableNamesSet new_dependencies = getDependenciesSetFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), it->second);
DatabaseCatalog::instance().updateLoadingDependencies(table_id, std::move(new_dependencies)); auto new_dependencies = getLoadingDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), it->second);
DatabaseCatalog::instance().updateDependencies(table_id, new_dependencies);
} }
std::vector<std::pair<ASTPtr, StoragePtr>> DatabaseMemory::getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const std::vector<std::pair<ASTPtr, StoragePtr>> DatabaseMemory::getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const

View File

@ -4,7 +4,7 @@
#include <Databases/DatabaseOnDisk.h> #include <Databases/DatabaseOnDisk.h>
#include <Databases/DatabaseOrdinary.h> #include <Databases/DatabaseOrdinary.h>
#include <Databases/DatabasesCommon.h> #include <Databases/DatabasesCommon.h>
#include <Databases/DDLDependencyVisitor.h> #include <Databases/DDLLoadingDependencyVisitor.h>
#include <Databases/TablesLoader.h> #include <Databases/TablesLoader.h>
#include <IO/ReadBufferFromFile.h> #include <IO/ReadBufferFromFile.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
@ -205,21 +205,9 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables
} }
QualifiedTableName qualified_name{TSA_SUPPRESS_WARNING_FOR_READ(database_name), create_query->getTable()}; QualifiedTableName qualified_name{TSA_SUPPRESS_WARNING_FOR_READ(database_name), create_query->getTable()};
TableNamesSet loading_dependencies = getDependenciesSetFromCreateQuery(getContext(), qualified_name, ast);
std::lock_guard lock{metadata.mutex}; std::lock_guard lock{metadata.mutex};
metadata.parsed_tables[qualified_name] = ParsedTableMetadata{full_path.string(), ast}; metadata.parsed_tables[qualified_name] = ParsedTableMetadata{full_path.string(), ast};
if (loading_dependencies.empty())
{
metadata.independent_database_objects.emplace_back(std::move(qualified_name));
}
else
{
for (const auto & dependency : loading_dependencies)
metadata.dependencies_info[dependency].dependent_database_objects.insert(qualified_name);
assert(metadata.dependencies_info[qualified_name].dependencies.empty());
metadata.dependencies_info[qualified_name].dependencies = std::move(loading_dependencies);
}
metadata.total_dictionaries += create_query->is_dictionary; metadata.total_dictionaries += create_query->is_dictionary;
} }
} }
@ -321,8 +309,8 @@ void DatabaseOrdinary::alterTable(ContextPtr local_context, const StorageID & ta
out.close(); out.close();
} }
TableNamesSet new_dependencies = getDependenciesSetFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), ast); auto new_dependencies = getLoadingDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), ast);
DatabaseCatalog::instance().updateLoadingDependencies(table_id, std::move(new_dependencies)); DatabaseCatalog::instance().updateDependencies(table_id, new_dependencies);
commitAlterTable(table_id, table_metadata_tmp_path, table_metadata_path, statement, local_context); commitAlterTable(table_id, table_metadata_tmp_path, table_metadata_path, statement, local_context);
} }

View File

@ -702,7 +702,18 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
/// We will drop or move tables which exist only in local metadata /// We will drop or move tables which exist only in local metadata
Strings tables_to_detach; Strings tables_to_detach;
std::vector<std::pair<String, String>> replicated_tables_to_rename;
struct RenameEdge
{
String from;
String intermediate;
String to;
};
/// This is needed to generate intermediate name
String salt = toString(thread_local_rng());
std::vector<RenameEdge> replicated_tables_to_rename;
size_t total_tables = 0; size_t total_tables = 0;
std::vector<UUID> replicated_ids; std::vector<UUID> replicated_ids;
for (auto existing_tables_it = getTablesIterator(getContext(), {}); existing_tables_it->isValid(); for (auto existing_tables_it = getTablesIterator(getContext(), {}); existing_tables_it->isValid();
@ -719,8 +730,15 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
{ {
if (name != it->second) if (name != it->second)
{ {
String intermediate_name;
/// Possibly we failed to rename it on previous iteration
/// And this table was already renamed to an intermediate name
if (startsWith(name, ".rename-") && !startsWith(it->second, ".rename-"))
intermediate_name = name;
else
intermediate_name = fmt::format(".rename-{}-{}", name, sipHash64(fmt::format("{}-{}", name, salt)));
/// Need just update table name /// Need just update table name
replicated_tables_to_rename.emplace_back(name, it->second); replicated_tables_to_rename.push_back({name, intermediate_name, it->second});
} }
continue; continue;
} }
@ -840,13 +858,13 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
tables_to_detach.size(), dropped_dictionaries, dropped_tables.size() - dropped_dictionaries, moved_tables); tables_to_detach.size(), dropped_dictionaries, dropped_tables.size() - dropped_dictionaries, moved_tables);
/// Now database is cleared from outdated tables, let's rename ReplicatedMergeTree tables to actual names /// Now database is cleared from outdated tables, let's rename ReplicatedMergeTree tables to actual names
for (const auto & old_to_new : replicated_tables_to_rename) /// We have to take into account that tables names could be changed with two general queries
/// 1) RENAME TABLE. There could be multiple pairs of tables (e.g. RENAME b TO c, a TO b, c TO d)
/// But it is equal to multiple subsequent RENAMEs each of which operates only with two tables
/// 2) EXCHANGE TABLE. This query swaps two names atomically and could not be represented with two separate RENAMEs
auto rename_table = [&](String from, String to)
{ {
const String & from = old_to_new.first;
const String & to = old_to_new.second;
LOG_DEBUG(log, "Will RENAME TABLE {} TO {}", backQuoteIfNeed(from), backQuoteIfNeed(to)); LOG_DEBUG(log, "Will RENAME TABLE {} TO {}", backQuoteIfNeed(from), backQuoteIfNeed(to));
/// TODO Maybe we should do it in two steps: rename all tables to temporary names and then rename them to actual names?
DDLGuardPtr table_guard = DatabaseCatalog::instance().getDDLGuard(db_name, std::min(from, to)); DDLGuardPtr table_guard = DatabaseCatalog::instance().getDDLGuard(db_name, std::min(from, to));
DDLGuardPtr to_table_guard = DatabaseCatalog::instance().getDDLGuard(db_name, std::max(from, to)); DDLGuardPtr to_table_guard = DatabaseCatalog::instance().getDDLGuard(db_name, std::max(from, to));
@ -858,7 +876,23 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
DatabaseAtomic::renameTable(make_query_context(), from, *this, to, false, false); DatabaseAtomic::renameTable(make_query_context(), from, *this, to, false, false);
tables_metadata_digest = new_digest; tables_metadata_digest = new_digest;
assert(checkDigestValid(getContext())); assert(checkDigestValid(getContext()));
};
LOG_DEBUG(log, "Starting first stage of renaming process. Will rename tables to intermediate names");
for (auto & [from, intermediate, _] : replicated_tables_to_rename)
{
/// Due to some unknown failures there could be tables
/// which are already in an intermediate state
/// For them we skip the first stage
if (from == intermediate)
continue;
rename_table(from, intermediate);
} }
LOG_DEBUG(log, "Starting second stage of renaming process. Will rename tables from intermediate to desired names");
for (auto & [_, intermediate, to] : replicated_tables_to_rename)
rename_table(intermediate, to);
LOG_DEBUG(log, "Renames completed succesessfully");
for (const auto & id : dropped_tables) for (const auto & id : dropped_tables)
DatabaseCatalog::instance().waitTableFinallyDropped(id); DatabaseCatalog::instance().waitTableFinallyDropped(id);

View File

@ -0,0 +1,56 @@
#include <Databases/NormalizeAndEvaluateConstantsVisitor.h>
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
namespace DB
{
void NormalizeAndEvaluateConstants::visit(const ASTPtr & ast, Data & data)
{
assert(data.create_query_context->hasQueryContext());
/// Looking for functions in column default expressions and dictionary source definition
if (const auto * function = ast->as<ASTFunction>())
visit(*function, data);
else if (const auto * dict_source = ast->as<ASTFunctionWithKeyValueArguments>())
visit(*dict_source, data);
}
void NormalizeAndEvaluateConstants::visit(const ASTFunction & function, Data & data)
{
/// Replace expressions like "dictGet(currentDatabase() || '.dict', 'value', toUInt32(1))"
/// with "dictGet('db_name.dict', 'value', toUInt32(1))"
ssize_t table_name_arg_idx = getPositionOfTableNameArgument(function);
if (table_name_arg_idx < 0)
return;
if (!function.arguments || function.arguments->children.size() <= static_cast<size_t>(table_name_arg_idx))
return;
auto & arg = function.arguments->as<ASTExpressionList &>().children[table_name_arg_idx];
if (arg->as<ASTFunction>())
arg = evaluateConstantExpressionAsLiteral(arg, data.create_query_context);
}
void NormalizeAndEvaluateConstants::visit(const ASTFunctionWithKeyValueArguments & dict_source, Data & data)
{
if (!dict_source.elements)
return;
auto & expr_list = dict_source.elements->as<ASTExpressionList &>();
for (auto & child : expr_list.children)
{
ASTPair * pair = child->as<ASTPair>();
if (pair->second->as<ASTFunction>())
{
auto ast_literal = evaluateConstantExpressionAsLiteral(pair->children[0], data.create_query_context);
pair->replace(pair->second, ast_literal);
}
}
}
}

View File

@ -0,0 +1,32 @@
#pragma once
#include <Core/QualifiedTableName.h>
#include <Parsers/IAST_fwd.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Databases/DDLLoadingDependencyVisitor.h>
namespace DB
{
/// Evaluates constants in DDL query.
class NormalizeAndEvaluateConstants : public DDLMatcherBase
{
public:
struct Data
{
ContextPtr create_query_context;
};
using Visitor = ConstInDepthNodeVisitor<NormalizeAndEvaluateConstants, true>;
static void visit(const ASTPtr & ast, Data & data);
private:
static void visit(const ASTFunction & function, Data & data);
static void visit(const ASTFunctionWithKeyValueArguments & dict_source, Data & data);
};
using NormalizeAndEvaluateConstantsVisitor = NormalizeAndEvaluateConstants::Visitor;
}

View File

@ -0,0 +1,659 @@
#include <Databases/TablesDependencyGraph.h>
#include <Common/logger_useful.h>
#include <boost/range/adaptor/reversed.hpp>
namespace DB
{
namespace ErrorCodes
{
extern const int INFINITE_LOOP;
}
namespace
{
constexpr const size_t CYCLIC_LEVEL = static_cast<size_t>(-2);
}
TablesDependencyGraph::TablesDependencyGraph(const String & name_for_logging_)
: name_for_logging(name_for_logging_)
{
}
TablesDependencyGraph::TablesDependencyGraph(const TablesDependencyGraph & src)
: TablesDependencyGraph(src.name_for_logging)
{
*this = src;
}
TablesDependencyGraph::TablesDependencyGraph(TablesDependencyGraph && src) noexcept
: TablesDependencyGraph(src.name_for_logging)
{
*this = std::move(src);
}
TablesDependencyGraph & TablesDependencyGraph::operator=(const TablesDependencyGraph & src)
{
if (&src != this)
{
nodes = src.nodes;
nodes_by_database_and_table_names = src.nodes_by_database_and_table_names;
nodes_by_uuid = src.nodes_by_uuid;
levels_calculated = src.levels_calculated;
nodes_sorted_by_level_lazy = src.nodes_sorted_by_level_lazy;
}
return *this;
}
TablesDependencyGraph & TablesDependencyGraph::operator=(TablesDependencyGraph && src) noexcept
{
nodes = std::exchange(src.nodes, decltype(nodes){});
nodes_by_database_and_table_names = std::exchange(src.nodes_by_database_and_table_names, decltype(nodes_by_database_and_table_names){});
nodes_by_uuid = std::exchange(src.nodes_by_uuid, decltype(nodes_by_uuid){});
levels_calculated = std::exchange(src.levels_calculated, false);
nodes_sorted_by_level_lazy = std::exchange(src.nodes_sorted_by_level_lazy, decltype(nodes_sorted_by_level_lazy){});
return *this;
}
void TablesDependencyGraph::clear()
{
nodes.clear();
nodes_by_database_and_table_names.clear();
nodes_by_uuid.clear();
setNeedRecalculateLevels();
}
bool TablesDependencyGraph::empty() const
{
return nodes.empty();
}
size_t TablesDependencyGraph::getNumberOfTables() const
{
return nodes.size();
}
void TablesDependencyGraph::addDependency(const StorageID & table_id, const StorageID & dependency)
{
auto * table_node = addOrUpdateNode(table_id);
auto * dependency_node = addOrUpdateNode(dependency);
if (table_node->dependencies.contains(dependency_node))
return; /// Already have this dependency.
table_node->dependencies.insert(dependency_node);
dependency_node->dependents.insert(table_node);
setNeedRecalculateLevels();
}
void TablesDependencyGraph::addDependencies(const StorageID & table_id, const std::vector<StorageID> & dependencies)
{
auto * table_node = addOrUpdateNode(table_id);
std::unordered_set<Node *> new_dependency_nodes;
for (const auto & dependency : dependencies)
new_dependency_nodes.emplace(addOrUpdateNode(dependency));
if (table_node->dependencies == new_dependency_nodes)
return;
auto old_dependencies = getDependencies(*table_node);
auto old_dependency_nodes = std::move(table_node->dependencies);
if (!old_dependencies.empty())
{
LOG_WARNING(
getLogger(),
"Replacing outdated dependencies ({}) of {} with: {}",
fmt::join(old_dependencies, ", "),
table_id,
fmt::join(dependencies, ", "));
}
for (auto * dependency_node : old_dependency_nodes)
{
if (!new_dependency_nodes.contains(dependency_node))
dependency_node->dependents.erase(table_node);
}
for (auto * dependency_node : new_dependency_nodes)
{
if (!old_dependency_nodes.contains(dependency_node))
dependency_node->dependents.insert(table_node);
}
table_node->dependencies = std::move(new_dependency_nodes);
setNeedRecalculateLevels();
}
void TablesDependencyGraph::addDependencies(const StorageID & table_id, const TableNamesSet & dependencies)
{
std::vector<StorageID> converted_dependencies;
for (const auto & dependency : dependencies)
converted_dependencies.emplace_back(StorageID{dependency});
addDependencies(table_id, converted_dependencies);
}
void TablesDependencyGraph::addDependencies(const QualifiedTableName & table_name, const TableNamesSet & dependencies)
{
addDependencies(StorageID{table_name}, dependencies);
}
bool TablesDependencyGraph::removeDependency(const StorageID & table_id, const StorageID & dependency, bool remove_isolated_tables)
{
auto * table_node = findNode(table_id);
if (!table_node)
return false;
auto * dependency_node = findNode(dependency);
if (!dependency_node)
return false;
auto dependency_it = table_node->dependencies.find(dependency_node);
if (dependency_it == table_node->dependencies.end())
return false;
table_node->dependencies.erase(dependency_it);
dependency_node->dependents.erase(table_node);
bool table_node_removed = false;
if (remove_isolated_tables && dependency_node->dependencies.empty() && dependency_node->dependents.empty())
{
removeNode(dependency_node);
if (table_node == dependency_node)
table_node_removed = true;
}
if (remove_isolated_tables && !table_node_removed && table_node->dependencies.empty() && table_node->dependents.empty())
removeNode(table_node);
setNeedRecalculateLevels();
return true;
}
std::vector<StorageID> TablesDependencyGraph::removeDependencies(const StorageID & table_id, bool remove_isolated_tables)
{
auto * table_node = findNode(table_id);
if (!table_node)
return {};
auto dependency_nodes = std::move(table_node->dependencies);
table_node->dependencies.clear();
bool table_node_removed = false;
std::vector<StorageID> dependencies;
dependencies.reserve(dependency_nodes.size());
for (auto * dependency_node : dependency_nodes)
{
dependencies.emplace_back(dependency_node->storage_id);
dependency_node->dependents.erase(table_node);
if (remove_isolated_tables && dependency_node->dependencies.empty() && dependency_node->dependents.empty())
{
removeNode(dependency_node);
if (table_node == dependency_node)
table_node_removed = true;
}
}
if (remove_isolated_tables && !table_node_removed && table_node->dependencies.empty() && table_node->dependents.empty())
removeNode(table_node);
setNeedRecalculateLevels();
return dependencies;
}
bool TablesDependencyGraph::removeTable(const StorageID & table_id)
{
auto * table_node = findNode(table_id);
if (!table_node)
return false;
removeNode(table_node);
setNeedRecalculateLevels();
return true;
}
TablesDependencyGraph::Node * TablesDependencyGraph::findNode(const StorageID & table_id) const
{
table_id.assertNotEmpty();
if (table_id.hasUUID())
{
auto it = nodes_by_uuid.find(table_id.uuid);
if (it != nodes_by_uuid.end())
return it->second; /// Found by UUID.
}
if (!table_id.table_name.empty())
{
auto it = nodes_by_database_and_table_names.find(table_id);
if (it != nodes_by_database_and_table_names.end())
{
auto * node = it->second;
if (table_id.hasUUID() && node->storage_id.hasUUID() && (table_id.uuid != node->storage_id.uuid))
return nullptr; /// UUID is different, it's not the node we're looking for.
return node; /// Found by table name.
}
}
return nullptr; /// Not found.
}
TablesDependencyGraph::Node * TablesDependencyGraph::addOrUpdateNode(const StorageID & table_id)
{
auto * node = findNode(table_id);
if (node)
{
/// Node has been found, maybe we can update the information in the graph with new table_name or new UUID.
if (table_id.hasUUID() && !node->storage_id.hasUUID())
{
node->storage_id.uuid = table_id.uuid;
nodes_by_uuid.emplace(node->storage_id.uuid, node);
}
if (!table_id.table_name.empty() && ((table_id.table_name != node->storage_id.table_name) || (table_id.database_name != node->storage_id.database_name)))
{
auto it = nodes_by_database_and_table_names.find(table_id);
if (it != nodes_by_database_and_table_names.end())
{
LOG_WARNING(getLogger(), "Name conflict in the graph having tables {} and {} while adding table {}. Will remove {} from the graph",
node->storage_id, it->second->storage_id, table_id, it->second->storage_id);
removeNode(it->second);
}
nodes_by_database_and_table_names.erase(node->storage_id);
node->storage_id.database_name = table_id.database_name;
node->storage_id.table_name = table_id.table_name;
nodes_by_database_and_table_names.emplace(node->storage_id, node);
}
}
else
{
/// Node has not been found by UUID or table name.
if (!table_id.table_name.empty())
{
auto it = nodes_by_database_and_table_names.find(table_id);
if (it != nodes_by_database_and_table_names.end())
{
LOG_WARNING(getLogger(), "Name conflict in the graph having table {} while adding table {}. Will remove {} from the graph",
it->second->storage_id, table_id, it->second->storage_id);
removeNode(it->second);
}
}
auto node_ptr = std::make_shared<Node>(table_id);
nodes.insert(node_ptr);
node = node_ptr.get();
if (table_id.hasUUID())
nodes_by_uuid.emplace(table_id.uuid, node);
if (!table_id.table_name.empty())
nodes_by_database_and_table_names.emplace(table_id, node);
}
return node;
}
void TablesDependencyGraph::removeNode(Node * node)
{
auto dependency_nodes = std::move(node->dependencies);
auto dependent_nodes = std::move(node->dependents);
if (node->storage_id.hasUUID())
nodes_by_uuid.erase(node->storage_id.uuid);
if (!node->storage_id.table_name.empty())
nodes_by_database_and_table_names.erase(node->storage_id);
for (auto * dependency_node : dependency_nodes)
dependency_node->dependents.erase(node);
for (auto * dependent_node : dependent_nodes)
dependent_node->dependencies.erase(node);
nodes.erase(node->shared_from_this());
}
size_t TablesDependencyGraph::removeTablesIf(const std::function<bool(const StorageID &)> & function)
{
size_t num_removed = 0;
auto it = nodes.begin();
while (it != nodes.end())
{
auto * current = (it++)->get();
if (function(current->storage_id))
{
StorageID storage_id = current->storage_id;
removeNode(current);
++num_removed;
}
}
if (num_removed)
setNeedRecalculateLevels();
return num_removed;
}
size_t TablesDependencyGraph::removeIsolatedTables()
{
size_t num_removed = 0;
auto it = nodes.begin();
while (it != nodes.end())
{
auto * current = (it++)->get();
if (current->dependencies.empty() && current->dependents.empty())
{
removeNode(current);
++num_removed;
}
}
if (num_removed)
setNeedRecalculateLevels();
return num_removed;
}
std::vector<StorageID> TablesDependencyGraph::getTables() const
{
std::vector<StorageID> res;
res.reserve(nodes.size());
for (const auto & node : nodes)
res.emplace_back(node->storage_id);
return res;
}
void TablesDependencyGraph::mergeWith(const TablesDependencyGraph & other)
{
for (const auto & other_node : other.nodes)
addDependencies(other_node->storage_id, other.getDependencies(*other_node));
}
std::vector<StorageID> TablesDependencyGraph::getDependencies(const StorageID & table_id) const
{
const auto * node = findNode(table_id);
if (!node)
return {};
return getDependencies(*node);
}
std::vector<StorageID> TablesDependencyGraph::getDependencies(const Node & node)
{
std::vector<StorageID> res;
res.reserve(node.dependencies.size());
for (const auto * dependency_node : node.dependencies)
res.emplace_back(dependency_node->storage_id);
return res;
}
size_t TablesDependencyGraph::getNumberOfDependencies(const StorageID & table_id) const
{
const auto * node = findNode(table_id);
if (!node)
return 0;
return node->dependencies.size();
}
std::vector<StorageID> TablesDependencyGraph::getDependents(const StorageID & table_id) const
{
const auto * node = findNode(table_id);
if (!node)
return {};
return getDependents(*node);
}
std::vector<StorageID> TablesDependencyGraph::getDependents(const Node & node)
{
std::vector<StorageID> res;
res.reserve(node.dependents.size());
for (const auto * dependent_node : node.dependents)
res.emplace_back(dependent_node->storage_id);
return res;
}
size_t TablesDependencyGraph::getNumberOfDependents(const StorageID & table_id) const
{
const auto * node = findNode(table_id);
if (!node)
return 0;
return node->dependents.size();
}
void TablesDependencyGraph::getNumberOfAdjacents(const StorageID & table_id, size_t & num_dependencies, size_t & num_dependents) const
{
num_dependencies = 0;
num_dependents = 0;
const auto * node = findNode(table_id);
if (!node)
return;
num_dependencies = node->dependencies.size();
num_dependents = node->dependents.size();
}
bool TablesDependencyGraph::isIsolatedTable(const StorageID & table_id) const
{
const auto * node = findNode(table_id);
if (!node)
return false;
return node->dependencies.empty() && node->dependents.empty();
}
void TablesDependencyGraph::checkNoCyclicDependencies() const
{
if (hasCyclicDependencies())
{
throw Exception(
ErrorCodes::INFINITE_LOOP,
"{}: Tables {} have cyclic dependencies: {}",
name_for_logging,
fmt::join(getTablesWithCyclicDependencies(), ", "),
describeCyclicDependencies());
}
}
bool TablesDependencyGraph::hasCyclicDependencies() const
{
const auto & nodes_sorted_by_level = getNodesSortedByLevel();
return !nodes_sorted_by_level.empty() && (nodes_sorted_by_level.back()->level == CYCLIC_LEVEL);
}
std::vector<StorageID> TablesDependencyGraph::getTablesWithCyclicDependencies() const
{
std::vector<StorageID> res;
for (const auto * node : getNodesSortedByLevel() | boost::adaptors::reversed)
{
if (node->level != CYCLIC_LEVEL)
break;
res.emplace_back(node->storage_id);
}
return res;
}
String TablesDependencyGraph::describeCyclicDependencies() const
{
String res;
for (const auto * node : getNodesSortedByLevel() | boost::adaptors::reversed)
{
if (node->level != CYCLIC_LEVEL)
break;
if (!res.empty())
res += "; ";
res += node->storage_id.getNameForLogs();
res += " -> [";
bool need_comma = false;
for (const auto * dependency_node : node->dependencies)
{
if (dependency_node->level != CYCLIC_LEVEL)
continue;
if (need_comma)
res += ", ";
need_comma = true;
res += dependency_node->storage_id.getNameForLogs();
}
res += "]";
}
return res;
}
void TablesDependencyGraph::setNeedRecalculateLevels()
{
levels_calculated = false;
nodes_sorted_by_level_lazy.clear();
}
void TablesDependencyGraph::calculateLevels() const
{
if (levels_calculated)
return;
levels_calculated = true;
nodes_sorted_by_level_lazy.clear();
nodes_sorted_by_level_lazy.reserve(nodes.size());
std::unordered_set<const Node *> nodes_to_process;
for (const auto & node_ptr : nodes)
nodes_to_process.emplace(node_ptr.get());
size_t current_level = 0;
while (!nodes_to_process.empty())
{
size_t old_num_sorted = nodes_sorted_by_level_lazy.size();
for (auto it = nodes_to_process.begin(); it != nodes_to_process.end();)
{
const auto * current_node = *(it++);
bool has_dependencies = false;
for (const auto * dependency : current_node->dependencies)
{
if (nodes_to_process.contains(dependency))
has_dependencies = true;
}
if (!has_dependencies)
{
current_node->level = current_level;
nodes_sorted_by_level_lazy.emplace_back(current_node);
}
}
if (nodes_sorted_by_level_lazy.size() == old_num_sorted)
break;
for (size_t i = old_num_sorted; i != nodes_sorted_by_level_lazy.size(); ++i)
nodes_to_process.erase(nodes_sorted_by_level_lazy[i]);
++current_level;
}
for (const auto * node_with_cyclic_dependencies : nodes_to_process)
{
node_with_cyclic_dependencies->level = CYCLIC_LEVEL;
nodes_sorted_by_level_lazy.emplace_back(node_with_cyclic_dependencies);
}
}
const TablesDependencyGraph::NodesSortedByLevel & TablesDependencyGraph::getNodesSortedByLevel() const
{
calculateLevels();
return nodes_sorted_by_level_lazy;
}
std::vector<StorageID> TablesDependencyGraph::getTablesSortedByDependency() const
{
std::vector<StorageID> res;
res.reserve(nodes.size());
for (const auto * node : getNodesSortedByLevel())
{
res.emplace_back(node->storage_id);
}
return res;
}
std::vector<std::vector<StorageID>> TablesDependencyGraph::getTablesSortedByDependencyForParallel() const
{
std::vector<std::vector<StorageID>> res;
std::optional<size_t> last_level;
for (const auto * node : getNodesSortedByLevel())
{
if (node->level != last_level)
res.emplace_back();
auto & table_ids = res.back();
table_ids.emplace_back(node->storage_id);
last_level = node->level;
}
return res;
}
void TablesDependencyGraph::log() const
{
if (empty())
{
LOG_TEST(getLogger(), "No tables");
return;
}
for (const auto * node : getNodesSortedByLevel())
{
String dependencies_desc = node->dependencies.empty()
? "no dependencies"
: fmt::format("{} dependencies: {}", node->dependencies.size(), fmt::join(getDependencies(*node), ", "));
String level_desc = (node->level == CYCLIC_LEVEL) ? "cyclic" : fmt::format("level {}", node->level);
LOG_TEST(getLogger(), "Table {} has {} ({})", node->storage_id, dependencies_desc, level_desc);
}
}
Poco::Logger * TablesDependencyGraph::getLogger() const
{
if (!logger)
logger = &Poco::Logger::get(name_for_logging);
return logger;
}
}

View File

@ -0,0 +1,171 @@
#pragma once
#include <Interpreters/StorageID.h>
#include <unordered_map>
#include <unordered_set>
namespace DB
{
using TableNamesSet = std::unordered_set<QualifiedTableName>;
/// Represents dependencies of some tables on other tables or dictionaries.
///
/// NOTES: A "dependent" depends on its "dependency". For example, if table "A" depends on table "B", then
/// "B" is a dependency for "A", and "A" is a dependent for "B".
///
/// Dependencies can be added to the graph in any order. For example, if table "A" depends on "B", and "B" depends on "C", then
/// it's allowed to add first "A->B" and then "B->C", or first "B->C" and then "A->B", the resulting graph will be the same.
///
/// This class is used to represent various types of table-table dependencies:
/// 1. View dependencies: "source_table -> materialized_view".
/// Data inserted to a source table is also inserted to corresponding materialized views.
/// 2. Loading dependencies: specify in which order tables must be loaded during startup.
/// For example a dictionary should be loaded after it's source table and it's written in the graph as "dictionary -> source_table".
/// 3. Referential dependencies: "table -> all tables mentioned in its definition".
/// Referential dependencies are checked to decide if it's safe to drop a table (it can be unsafe if the table is used by another table).
///
/// WARNING: This class doesn't have an embedded mutex, so it must be synchronized outside.
class TablesDependencyGraph
{
public:
explicit TablesDependencyGraph(const String & name_for_logging_);
TablesDependencyGraph(const TablesDependencyGraph & src);
TablesDependencyGraph(TablesDependencyGraph && src) noexcept;
TablesDependencyGraph & operator=(const TablesDependencyGraph & src);
TablesDependencyGraph & operator=(TablesDependencyGraph && src) noexcept;
/// The dependency graph is empty if doesn't contain any tables.
bool empty() const;
/// Clears this dependency graph.
void clear();
/// Adds a single dependency "table_id" on "dependency".
void addDependency(const StorageID & table_id, const StorageID & dependency);
/// Adds a table with specified dependencies if there are no dependencies of the table in the graph yet;
/// otherwise it replaces the dependencies of the table in the graph and shows a warning.
void addDependencies(const StorageID & table_id, const std::vector<StorageID> & dependencies);
void addDependencies(const StorageID & table_id, const TableNamesSet & dependencies);
void addDependencies(const QualifiedTableName & table_name, const TableNamesSet & dependencies);
/// Removes a single dependency of "table_id" on "dependency".
/// If "remove_isolated_tables" is set the function will also remove tables with no dependencies and no dependents
/// from the graph.
bool removeDependency(const StorageID & table_id, const StorageID & dependency, bool remove_isolated_tables = false);
/// Removes all dependencies of "table_id", returns those dependencies.
std::vector<StorageID> removeDependencies(const StorageID & table_id, bool remove_isolated_tables = false);
/// Removes a table from the graph and removes all references to in from the graph (both from its dependencies and dependents).
bool removeTable(const StorageID & table_id);
/// Removes tables from the graph by a specified filter.
size_t removeTablesIf(const std::function<bool(const StorageID &)> & function);
/// Removes tables with no dependencies and no dependents from the graph.
size_t removeIsolatedTables();
/// Returns the number of tables in the graph.
size_t getNumberOfTables() const;
/// Returns a list of all tables in the graph.
std::vector<StorageID> getTables() const;
/// Adds tables and dependencies with another graph.
void mergeWith(const TablesDependencyGraph & other);
/// Returns a list of dependencies of a specified table.
std::vector<StorageID> getDependencies(const StorageID & table_id) const;
size_t getNumberOfDependencies(const StorageID & table_id) const;
bool hasDependencies(const StorageID & table_id) const { return getNumberOfDependencies(table_id) != 0; }
/// Returns a list of dependents of a specified table.
std::vector<StorageID> getDependents(const StorageID & table_id) const;
size_t getNumberOfDependents(const StorageID & table_id) const;
bool hasDependents(const StorageID & table_id) const { return getNumberOfDependents(table_id) != 0; }
/// Returns the number of dependencies and the number of dependents of a specified table.
void getNumberOfAdjacents(const StorageID & table_id, size_t & num_dependencies, size_t & num_dependents) const;
/// Returns true if a specified table has no dependencies and no dependents.
bool isIsolatedTable(const StorageID & table_id) const;
/// Checks that there are no cyclic dependencies in the graph.
/// Cyclic dependencies are dependencies like "A->A" or "A->B->C->D->A".
void checkNoCyclicDependencies() const;
bool hasCyclicDependencies() const;
std::vector<StorageID> getTablesWithCyclicDependencies() const;
String describeCyclicDependencies() const;
/// Returns a list of tables sorted by their dependencies:
/// tables without dependencies first, then
/// tables which depend on the tables without dependencies, then
/// tables which depend on the tables which depend on the tables without dependencies, and so on.
std::vector<StorageID> getTablesSortedByDependency() const;
/// The same as getTablesSortedByDependency() but make a list for parallel processing.
std::vector<std::vector<StorageID>> getTablesSortedByDependencyForParallel() const;
/// Outputs information about this graph as a bunch of logging messages.
void log() const;
private:
struct Node : public std::enable_shared_from_this<Node>
{
StorageID storage_id;
/// If A depends on B then "A.dependencies" contains "B".
std::unordered_set<Node *> dependencies;
/// If A depends on B then "B.dependents" contains "A".
std::unordered_set<Node *> dependents;
/// Tables without dependencies have level == 0, tables which depend on the tables without dependencies have level == 1, and so on.
/// Calculated lazily.
mutable size_t level = 0;
explicit Node(const StorageID & storage_id_) : storage_id(storage_id_) {}
};
using NodeSharedPtr = std::shared_ptr<Node>;
struct LessByLevel
{
bool operator()(const Node * left, const Node * right) { return left->level < right->level; }
};
std::unordered_set<NodeSharedPtr> nodes;
/// Nodes can be found either by UUID or by database name & table name. That's why we need two maps here.
std::unordered_map<StorageID, Node *, StorageID::DatabaseAndTableNameHash, StorageID::DatabaseAndTableNameEqual> nodes_by_database_and_table_names;
std::unordered_map<UUID, Node *> nodes_by_uuid;
/// This is set if both `level` inside each node and `nodes_sorted_by_level_lazy` are calculated.
mutable bool levels_calculated = false;
/// Nodes sorted by their level. Calculated lazily.
using NodesSortedByLevel = std::vector<const Node *>;
mutable NodesSortedByLevel nodes_sorted_by_level_lazy;
const String name_for_logging;
mutable Poco::Logger * logger = nullptr;
Node * findNode(const StorageID & table_id) const;
Node * addOrUpdateNode(const StorageID & table_id);
void removeNode(Node * node);
static std::vector<StorageID> getDependencies(const Node & node);
static std::vector<StorageID> getDependents(const Node & node);
void setNeedRecalculateLevels();
void calculateLevels() const;
const NodesSortedByLevel & getNodesSortedByLevel() const;
Poco::Logger * getLogger() const;
};
}

View File

@ -1,6 +1,7 @@
#include <Databases/TablesLoader.h> #include <Databases/TablesLoader.h>
#include <Databases/IDatabase.h> #include <Databases/IDatabase.h>
#include <Databases/DDLDependencyVisitor.h> #include <Databases/DDLDependencyVisitor.h>
#include <Databases/DDLLoadingDependencyVisitor.h>
#include <Interpreters/DatabaseCatalog.h> #include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/ExternalDictionariesLoader.h> #include <Interpreters/ExternalDictionariesLoader.h>
@ -14,45 +15,12 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int INFINITE_LOOP;
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
static constexpr size_t PRINT_MESSAGE_EACH_N_OBJECTS = 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 PRINT_MESSAGE_EACH_N_SECONDS = 5;
void mergeDependenciesGraphs(DependenciesInfos & main_dependencies_info, const DependenciesInfos & additional_info)
{
for (const auto & table_and_info : additional_info)
{
const QualifiedTableName & table = table_and_info.first;
const TableNamesSet & dependent_tables = table_and_info.second.dependent_database_objects;
const TableNamesSet & dependencies = table_and_info.second.dependencies;
DependenciesInfo & maybe_existing_info = main_dependencies_info[table];
maybe_existing_info.dependent_database_objects.insert(dependent_tables.begin(), dependent_tables.end());
if (!dependencies.empty())
{
if (maybe_existing_info.dependencies.empty())
maybe_existing_info.dependencies = dependencies;
else if (maybe_existing_info.dependencies != dependencies)
{
/// Can happen on DatabaseReplicated recovery
LOG_WARNING(&Poco::Logger::get("TablesLoader"), "Replacing outdated dependencies ({}) of {} with: {}",
fmt::join(maybe_existing_info.dependencies, ", "),
table,
fmt::join(dependencies, ", "));
for (const auto & old_dependency : maybe_existing_info.dependencies)
{
[[maybe_unused]] bool removed = main_dependencies_info[old_dependency].dependent_database_objects.erase(table);
assert(removed);
}
maybe_existing_info.dependencies = dependencies;
}
}
}
}
void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, AtomicStopwatch & watch) 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)) if (processed % PRINT_MESSAGE_EACH_N_OBJECTS == 0 || watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS))
@ -66,6 +34,8 @@ TablesLoader::TablesLoader(ContextMutablePtr global_context_, Databases database
: global_context(global_context_) : global_context(global_context_)
, databases(std::move(databases_)) , databases(std::move(databases_))
, strictness_mode(strictness_mode_) , strictness_mode(strictness_mode_)
, referential_dependencies("ReferentialDeps")
, loading_dependencies("LoadingDeps")
{ {
metadata.default_database = global_context->getCurrentDatabase(); metadata.default_database = global_context->getCurrentDatabase();
log = &Poco::Logger::get("TablesLoader"); log = &Poco::Logger::get("TablesLoader");
@ -101,20 +71,18 @@ void TablesLoader::loadTables()
stopwatch.restart(); stopwatch.restart();
logDependencyGraph(); buildDependencyGraph();
/// Remove tables that do not exist
removeUnresolvableDependencies(/* remove_loaded */ false);
/// Update existing info (it's important for ATTACH DATABASE) /// Update existing info (it's important for ATTACH DATABASE)
DatabaseCatalog::instance().addLoadingDependencies(metadata.dependencies_info); DatabaseCatalog::instance().addDependencies(referential_dependencies);
/// Some tables were loaded by database with loadStoredObjects(...). Remove them from graph if necessary. /// Remove tables that do not exist
removeUnresolvableDependencies(/* remove_loaded */ true); removeUnresolvableDependencies();
loadTablesInTopologicalOrder(pool); loadTablesInTopologicalOrder(pool);
} }
void TablesLoader::startupTables() void TablesLoader::startupTables()
{ {
/// Startup tables after all tables are loaded. Background tasks (merges, mutations, etc) may slow down data parts loading. /// Startup tables after all tables are loaded. Background tasks (merges, mutations, etc) may slow down data parts loading.
@ -123,52 +91,79 @@ void TablesLoader::startupTables()
} }
void TablesLoader::removeUnresolvableDependencies(bool remove_loaded) void TablesLoader::buildDependencyGraph()
{ {
auto need_exclude_dependency = [this, remove_loaded](const QualifiedTableName & dependency_name, const DependenciesInfo & info) for (const auto & [table_name, table_metadata] : metadata.parsed_tables)
{
auto new_loading_dependencies = getLoadingDependenciesFromCreateQuery(global_context, table_name, table_metadata.ast);
if (!new_loading_dependencies.empty())
referential_dependencies.addDependencies(table_name, new_loading_dependencies);
/// We're adding `new_loading_dependencies` to the graph here even if they're empty because
/// we need to have all tables from `metadata.parsed_tables` in the graph.
loading_dependencies.addDependencies(table_name, new_loading_dependencies);
}
referential_dependencies.log();
loading_dependencies.log();
}
void TablesLoader::removeUnresolvableDependencies()
{
auto need_exclude_dependency = [this](const StorageID & table_id)
{ {
/// Table exists and will be loaded /// Table exists and will be loaded
if (metadata.parsed_tables.contains(dependency_name)) if (metadata.parsed_tables.contains(table_id.getQualifiedName()))
return false; return false;
/// Table exists and it's already loaded
if (DatabaseCatalog::instance().isTableExist(StorageID(dependency_name.database, dependency_name.table), global_context)) if (DatabaseCatalog::instance().isTableExist(table_id, global_context))
return remove_loaded;
/// It's XML dictionary.
if (dependency_name.database == metadata.default_database &&
global_context->getExternalDictionariesLoader().has(dependency_name.table))
{ {
LOG_WARNING(log, "Tables {} depend on XML dictionary {}, but XML dictionaries are loaded independently." /// Table exists and it's already loaded
"Consider converting it to DDL dictionary.", fmt::join(info.dependent_database_objects, ", "), dependency_name); }
return true; else if (table_id.database_name == metadata.default_database &&
global_context->getExternalDictionariesLoader().has(table_id.table_name))
{
/// Tables depend on a XML dictionary.
LOG_WARNING(
log,
"Tables {} depend on XML dictionary {}, but XML dictionaries are loaded independently."
"Consider converting it to DDL dictionary.",
fmt::join(loading_dependencies.getDependents(table_id), ", "),
table_id);
}
else
{
/// Some tables depend on table "table_id", but there is no such table in DatabaseCatalog and we don't have its metadata.
/// We will ignore it and try to load dependent tables without "table_id"
/// (but most likely dependent tables will fail to load).
LOG_WARNING(
log,
"Tables {} depend on {}, but seems like that does not exist. Will ignore it and try to load existing tables",
fmt::join(loading_dependencies.getDependents(table_id), ", "),
table_id);
} }
/// Some tables depends on table "dependency_name", but there is no such table in DatabaseCatalog and we don't have its metadata. size_t num_dependencies, num_dependents;
/// We will ignore it and try to load dependent tables without "dependency_name" loading_dependencies.getNumberOfAdjacents(table_id, num_dependencies, num_dependents);
/// (but most likely dependent tables will fail to load). if (num_dependencies || !num_dependents)
LOG_WARNING(log, "Tables {} depend on {}, but seems like the it does not exist. Will ignore it and try to load existing tables",
fmt::join(info.dependent_database_objects, ", "), dependency_name);
if (!info.dependencies.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {} does not exist, but we have seen its AST and found {} dependencies."
"It's a bug", dependency_name, info.dependencies.size());
if (info.dependent_database_objects.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {} does not have dependencies and dependent tables as it expected to." throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {} does not have dependencies and dependent tables as it expected to."
"It's a bug", dependency_name); "It's a bug", table_id);
return true; return true; /// Exclude this dependency.
}; };
auto table_it = metadata.dependencies_info.begin(); loading_dependencies.removeTablesIf(need_exclude_dependency);
while (table_it != metadata.dependencies_info.end())
{ if (loading_dependencies.getNumberOfTables() != metadata.parsed_tables.size())
auto & info = table_it->second; throw Exception(ErrorCodes::LOGICAL_ERROR, "Number of tables to be loaded is not as expected. It's a bug");
if (need_exclude_dependency(table_it->first, info))
table_it = removeResolvedDependency(table_it, metadata.independent_database_objects); /// Cannot load tables with cyclic dependencies.
else loading_dependencies.checkNoCyclicDependencies();
++table_it;
}
} }
void TablesLoader::loadTablesInTopologicalOrder(ThreadPool & pool) void TablesLoader::loadTablesInTopologicalOrder(ThreadPool & pool)
{ {
/// Compatibility setting which should be enabled by default on attach /// Compatibility setting which should be enabled by default on attach
@ -176,81 +171,25 @@ void TablesLoader::loadTablesInTopologicalOrder(ThreadPool & pool)
ContextMutablePtr load_context = Context::createCopy(global_context); ContextMutablePtr load_context = Context::createCopy(global_context);
load_context->setSetting("cast_ipv4_ipv6_default_on_conversion_error", 1); load_context->setSetting("cast_ipv4_ipv6_default_on_conversion_error", 1);
/// Load independent tables in parallel. /// Load tables in parallel.
/// Then remove loaded tables from dependency graph, find tables/dictionaries that do not have unresolved dependencies anymore, auto tables_to_load = loading_dependencies.getTablesSortedByDependencyForParallel();
/// move them to the list of independent tables and load.
/// Repeat until we have some tables to load.
/// If we do not, then either all objects are loaded or there is cyclic dependency.
/// Complexity: O(V + E)
size_t level = 0;
do
{
assert(metadata.parsed_tables.size() == tables_processed + metadata.independent_database_objects.size() + getNumberOfTablesWithDependencies());
logDependencyGraph();
startLoadingIndependentTables(pool, level, load_context); for (size_t level = 0; level != tables_to_load.size(); ++level)
TableNames new_independent_database_objects;
for (const auto & table_name : metadata.independent_database_objects)
{ {
auto info_it = metadata.dependencies_info.find(table_name); startLoadingTables(pool, load_context, tables_to_load[level], level);
if (info_it == metadata.dependencies_info.end())
{
/// No tables depend on table_name and it was not even added to dependencies_info
continue;
}
removeResolvedDependency(info_it, new_independent_database_objects);
}
pool.wait(); pool.wait();
metadata.independent_database_objects = std::move(new_independent_database_objects);
++level;
} while (!metadata.independent_database_objects.empty());
checkCyclicDependencies();
}
DependenciesInfosIter TablesLoader::removeResolvedDependency(const DependenciesInfosIter & info_it, TableNames & independent_database_objects)
{
const QualifiedTableName & table_name = info_it->first;
const DependenciesInfo & info = info_it->second;
if (!info.dependencies.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {} is in list of independent tables, but dependencies count is {}."
"It's a bug", table_name, info.dependencies.size());
if (info.dependent_database_objects.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {} does not have dependent tables. It's a bug", table_name);
/// Decrement number of dependencies for each dependent table
for (const auto & dependent_table : info.dependent_database_objects)
{
auto & dependent_info = metadata.dependencies_info[dependent_table];
auto & dependencies_set = dependent_info.dependencies;
if (dependencies_set.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to decrement 0 dependencies counter for {}. It's a bug", dependent_table);
if (!dependencies_set.erase(table_name))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot remove {} from dependencies set of {}, it contains only {}",
table_name, dependent_table, fmt::join(dependencies_set, ", "));
if (dependencies_set.empty())
{
independent_database_objects.push_back(dependent_table);
if (dependent_info.dependent_database_objects.empty())
metadata.dependencies_info.erase(dependent_table);
} }
} }
return metadata.dependencies_info.erase(info_it); void TablesLoader::startLoadingTables(ThreadPool & pool, ContextMutablePtr load_context, const std::vector<StorageID> & tables_to_load, size_t level)
}
void TablesLoader::startLoadingIndependentTables(ThreadPool & pool, size_t level, ContextMutablePtr load_context)
{ {
size_t total_tables = metadata.parsed_tables.size(); size_t total_tables = metadata.parsed_tables.size();
LOG_INFO(log, "Loading {} tables with {} dependency level", metadata.independent_database_objects.size(), level); LOG_INFO(log, "Loading {} tables with dependency level {}", tables_to_load.size(), level);
for (const auto & table_name : metadata.independent_database_objects) for (const auto & table_id : tables_to_load)
{ {
pool.scheduleOrThrowOnError([this, load_context, total_tables, &table_name]() pool.scheduleOrThrowOnError([this, load_context, total_tables, table_name = table_id.getQualifiedName()]()
{ {
const auto & path_and_query = metadata.parsed_tables[table_name]; const auto & path_and_query = metadata.parsed_tables[table_name];
databases[table_name.database]->loadTableFromMetadata(load_context, path_and_query.path, table_name, path_and_query.ast, strictness_mode); databases[table_name.database]->loadTableFromMetadata(load_context, path_and_query.path, table_name, path_and_query.ast, strictness_mode);
@ -259,47 +198,4 @@ void TablesLoader::startLoadingIndependentTables(ThreadPool & pool, size_t level
} }
} }
size_t TablesLoader::getNumberOfTablesWithDependencies() const
{
size_t number_of_tables_with_dependencies = 0;
for (const auto & info : metadata.dependencies_info)
if (!info.second.dependencies.empty())
++number_of_tables_with_dependencies;
return number_of_tables_with_dependencies;
}
void TablesLoader::checkCyclicDependencies() const
{
/// Loading is finished if all dependencies are resolved
if (metadata.dependencies_info.empty())
return;
for (const auto & info : metadata.dependencies_info)
{
LOG_WARNING(log, "Cannot resolve dependencies: Table {} have {} dependencies and {} dependent tables. List of dependent tables: {}",
info.first, info.second.dependencies.size(),
info.second.dependent_database_objects.size(), fmt::join(info.second.dependent_database_objects, ", "));
assert(info.second.dependencies.empty());
}
throw Exception(ErrorCodes::INFINITE_LOOP, "Cannot attach {} tables due to cyclic dependencies. "
"See server log for details.", metadata.dependencies_info.size());
}
void TablesLoader::logDependencyGraph() const
{
LOG_TEST(log, "Have {} independent tables: {}",
metadata.independent_database_objects.size(),
fmt::join(metadata.independent_database_objects, ", "));
for (const auto & dependencies : metadata.dependencies_info)
{
LOG_TEST(log,
"Table {} have {} dependencies and {} dependent tables. List of dependent tables: {}",
dependencies.first,
dependencies.second.dependencies.size(),
dependencies.second.dependent_database_objects.size(),
fmt::join(dependencies.second.dependent_database_objects, ", "));
}
}
} }

View File

@ -6,6 +6,7 @@
#include <Core/QualifiedTableName.h> #include <Core/QualifiedTableName.h>
#include <Core/Types.h> #include <Core/Types.h>
#include <Databases/LoadingStrictnessLevel.h> #include <Databases/LoadingStrictnessLevel.h>
#include <Databases/TablesDependencyGraph.h>
#include <Interpreters/Context_fwd.h> #include <Interpreters/Context_fwd.h>
#include <Parsers/IAST_fwd.h> #include <Parsers/IAST_fwd.h>
#include <Common/Stopwatch.h> #include <Common/Stopwatch.h>
@ -34,21 +35,6 @@ struct ParsedTableMetadata
}; };
using ParsedMetadata = std::map<QualifiedTableName, ParsedTableMetadata>; using ParsedMetadata = std::map<QualifiedTableName, ParsedTableMetadata>;
using TableNames = std::vector<QualifiedTableName>;
using TableNamesSet = std::unordered_set<QualifiedTableName>;
struct DependenciesInfo
{
/// Set of dependencies
TableNamesSet dependencies;
/// Set of tables/dictionaries which depend on this table/dictionary
TableNamesSet dependent_database_objects;
};
using DependenciesInfos = std::unordered_map<QualifiedTableName, DependenciesInfo>;
using DependenciesInfosIter = std::unordered_map<QualifiedTableName, DependenciesInfo>::iterator;
void mergeDependenciesGraphs(DependenciesInfos & main_dependencies_info, const DependenciesInfos & additional_info);
struct ParsedTablesMetadata struct ParsedTablesMetadata
{ {
@ -59,17 +45,6 @@ struct ParsedTablesMetadata
/// For logging /// For logging
size_t total_dictionaries = 0; size_t total_dictionaries = 0;
/// List of tables/dictionaries that do not have any dependencies and can be loaded
TableNames independent_database_objects;
/// Adjacent list of dependency graph, contains two maps
/// 2. table/dictionary name -> dependent tables/dictionaries list (adjacency list of dependencies graph).
/// 1. table/dictionary name -> dependencies of table/dictionary (adjacency list of inverted dependencies graph)
/// If table A depends on table B, then there is an edge B --> A, i.e. dependencies_info[B].dependent_database_objects contains A
/// and dependencies_info[A].dependencies contain B.
/// We need inverted graph to effectively maintain it on DDL queries that can modify the graph.
DependenciesInfos dependencies_info;
}; };
/// Loads tables (and dictionaries) from specified databases /// Loads tables (and dictionaries) from specified databases
@ -92,25 +67,18 @@ private:
Strings databases_to_load; Strings databases_to_load;
ParsedTablesMetadata metadata; ParsedTablesMetadata metadata;
TablesDependencyGraph referential_dependencies;
TablesDependencyGraph loading_dependencies;
Poco::Logger * log; Poco::Logger * log;
std::atomic<size_t> tables_processed{0}; std::atomic<size_t> tables_processed{0};
AtomicStopwatch stopwatch; AtomicStopwatch stopwatch;
ThreadPool pool; ThreadPool pool;
void removeUnresolvableDependencies(bool remove_loaded); void buildDependencyGraph();
void removeUnresolvableDependencies();
void loadTablesInTopologicalOrder(ThreadPool & pool); void loadTablesInTopologicalOrder(ThreadPool & pool);
void startLoadingTables(ThreadPool & pool, ContextMutablePtr load_context, const std::vector<StorageID> & tables_to_load, size_t level);
DependenciesInfosIter removeResolvedDependency(const DependenciesInfosIter & info_it, TableNames & independent_database_objects);
void startLoadingIndependentTables(ThreadPool & pool, size_t level, ContextMutablePtr load_context);
void checkCyclicDependencies() const;
size_t getNumberOfTablesWithDependencies() const;
void logDependencyGraph() const;
}; };
} }

View File

@ -480,7 +480,8 @@ void S3ObjectStorage::copyObjectImpl(
auto outcome = client_ptr->CopyObject(request); auto outcome = client_ptr->CopyObject(request);
if (!outcome.IsSuccess() && outcome.GetError().GetExceptionName() == "EntityTooLarge") if (!outcome.IsSuccess() && (outcome.GetError().GetExceptionName() == "EntityTooLarge"
|| outcome.GetError().GetExceptionName() == "InvalidRequest"))
{ // Can't come here with MinIO, MinIO allows single part upload for large objects. { // Can't come here with MinIO, MinIO allows single part upload for large objects.
copyObjectMultipartImpl(src_bucket, src_key, dst_bucket, dst_key, head, metadata); copyObjectMultipartImpl(src_bucket, src_key, dst_bucket, dst_key, head, metadata);
return; return;

View File

@ -180,6 +180,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
format_settings.try_infer_datetimes = settings.input_format_try_infer_datetimes; format_settings.try_infer_datetimes = settings.input_format_try_infer_datetimes;
format_settings.bson.output_string_as_string = settings.output_format_bson_string_as_string; format_settings.bson.output_string_as_string = settings.output_format_bson_string_as_string;
format_settings.bson.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_bson_skip_fields_with_unsupported_types_in_schema_inference; format_settings.bson.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_bson_skip_fields_with_unsupported_types_in_schema_inference;
format_settings.max_binary_string_size = settings.format_binary_max_string_size;
/// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context /// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context
if (format_settings.schema.is_server) if (format_settings.schema.is_server)

View File

@ -79,6 +79,8 @@ struct FormatSettings
UInt64 input_allow_errors_num = 0; UInt64 input_allow_errors_num = 0;
Float32 input_allow_errors_ratio = 0; Float32 input_allow_errors_ratio = 0;
UInt64 max_binary_string_size = 0;
struct struct
{ {
UInt64 row_group_size = 1000000; UInt64 row_group_size = 1000000;

View File

@ -31,7 +31,17 @@ void UserDefinedSQLFunctionVisitor::visit(ASTPtr & ast)
auto * old_value = child.get(); auto * old_value = child.get();
visit(child); visit(child);
ast->setOrReplace(old_value, child);
// child did not change
if (old_value == child.get())
return;
// child changed, we need to modify it in the list of children of the parent also
for (auto & current_child : ast->children)
{
if (current_child.get() == old_value)
current_child = child;
}
}; };
if (auto * col_decl = ast->as<ASTColumnDeclaration>()) if (auto * col_decl = ast->as<ASTColumnDeclaration>())

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