Merge branch 'master' into logical-error-race-tmp-dirs

This commit is contained in:
Sema Checherinda 2022-12-04 18:18:42 +01:00 committed by GitHub
commit 005cded791
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
196 changed files with 3829 additions and 1572 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

@ -668,7 +668,7 @@ log_query_views=1
## log_formatted_queries {#settings-log-formatted-queries} ## log_formatted_queries {#settings-log-formatted-queries}
Allows to log formatted queries to the [system.query_log](../../operations/system-tables/query_log.md) system table (populates `formatted_query` column in the [system.query_log](../../operations/system-tables/query_log.md)). Allows to log formatted queries to the [system.query_log](../../operations/system-tables/query_log.md) system table (populates `formatted_query` column in the [system.query_log](../../operations/system-tables/query_log.md)).
Possible values: Possible values:
@ -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

@ -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

@ -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,21 +625,62 @@ 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.
auto tables_to_create = tables_dependencies.getTablesSortedByDependency();
for (const auto & table_id : tables_to_create)
{ {
/// We need to create tables considering their dependencies. auto table_name = table_id.getQualifiedName();
auto tables_to_create = findTablesWithoutDependencies(); createTable(table_name);
if (tables_to_create.empty()) checkTable(table_name);
break; /// We've already created all the tables. insertDataToTable(table_name);
for (const auto & table_name : tables_to_create)
{
createTable(table_name);
checkTable(table_name);
insertDataToTable(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

@ -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

@ -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.
@ -523,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

@ -117,7 +117,7 @@ public:
WriteBuffer * working_buf = compressed_buffer ? compressed_buffer->getNestedBuffer() : file_buf.get(); WriteBuffer * working_buf = compressed_buffer ? compressed_buffer->getNestedBuffer() : file_buf.get();
/// Flush working buffer to file system /// Flush working buffer to file system
working_buf->next(); working_buf->next();
/// Fsync file system if needed /// Fsync file system if needed
@ -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,50 +547,96 @@ 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)
{ {
/// 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();
if (go_to_previous_file)
{ {
auto index_changelog = existing_changelogs.lower_bound(index); std::lock_guard lock(writer_mutex);
/// 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();
ChangelogFileDescription description; if (go_to_previous_file)
if (index_changelog->first == index) /// exactly this file starts from index
description = index_changelog->second;
else
description = std::prev(index_changelog)->second;
/// Initialize writer from this log file
current_writer = std::make_unique<ChangelogWriter>(description.path, WriteMode::Append, index_changelog->first);
/// Remove all subsequent files if overwritten something in previous one
auto to_remove_itr = existing_changelogs.upper_bound(index);
for (auto itr = to_remove_itr; itr != existing_changelogs.end();)
{ {
std::filesystem::remove(itr->second.path); auto index_changelog = existing_changelogs.lower_bound(index);
itr = existing_changelogs.erase(itr);
ChangelogFileDescription description;
if (index_changelog->first == index) /// exactly this file starts from index
description = index_changelog->second;
else
description = std::prev(index_changelog)->second;
/// Initialize writer from this log file
current_writer = std::make_unique<ChangelogWriter>(description.path, WriteMode::Append, index_changelog->first);
/// Remove all subsequent files if overwritten something in previous one
auto to_remove_itr = existing_changelogs.upper_bound(index);
for (auto itr = to_remove_itr; itr != existing_changelogs.end();)
{
std::filesystem::remove(itr->second.path);
itr = existing_changelogs.erase(itr);
}
} }
} }
@ -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

@ -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

@ -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"; {
DB::KeeperLogStore changelog1("./logs", 100, true, params.enable_compression); LOG_INFO(log, "================First time=====================");
changelog1.init(1, 0); DB::KeeperLogStore changelog1("./logs", 100, true, params.enable_compression);
auto entry = getLogEntry("hello_world", 1000); changelog1.init(1, 0);
changelog1.append(entry); auto entry = getLogEntry("hello_world", 1000);
changelog1.end_of_append_batch(0, 0); changelog1.append(entry);
EXPECT_EQ(changelog1.next_slot(), 2); changelog1.end_of_append_batch(0, 0);
EXPECT_EQ(changelog1.next_slot(), 2);
waitDurableLogs(changelog1);
}
std::cerr << "================Second time=====================\n"; {
DB::KeeperLogStore changelog2("./logs", 100, true, params.enable_compression); LOG_INFO(log, "================Second time=====================");
changelog2.init(1, 0); DB::KeeperLogStore changelog2("./logs", 100, true, params.enable_compression);
entry = getLogEntry("hello_world", 1000); changelog2.init(1, 0);
changelog2.append(entry); auto entry = getLogEntry("hello_world", 1000);
changelog2.end_of_append_batch(0, 0); changelog2.append(entry);
EXPECT_EQ(changelog2.next_slot(), 3); changelog2.end_of_append_batch(0, 0);
EXPECT_EQ(changelog2.next_slot(), 3);
waitDurableLogs(changelog2);
}
std::cerr << "================Third time=====================\n"; {
DB::KeeperLogStore changelog3("./logs", 100, true, params.enable_compression); LOG_INFO(log, "================Third time=====================");
changelog3.init(1, 0); DB::KeeperLogStore changelog3("./logs", 100, true, params.enable_compression);
entry = getLogEntry("hello_world", 1000); changelog3.init(1, 0);
changelog3.append(entry); auto entry = getLogEntry("hello_world", 1000);
changelog3.end_of_append_batch(0, 0); changelog3.append(entry);
EXPECT_EQ(changelog3.next_slot(), 4); changelog3.end_of_append_batch(0, 0);
EXPECT_EQ(changelog3.next_slot(), 4);
waitDurableLogs(changelog3);
}
std::cerr << "================Fourth time=====================\n"; {
DB::KeeperLogStore changelog4("./logs", 100, true, params.enable_compression); LOG_INFO(log, "================Fourth time=====================");
changelog4.init(1, 0); DB::KeeperLogStore changelog4("./logs", 100, true, params.enable_compression);
entry = getLogEntry("hello_world", 1000); changelog4.init(1, 0);
changelog4.append(entry); auto entry = getLogEntry("hello_world", 1000);
changelog4.end_of_append_batch(0, 0); changelog4.append(entry);
EXPECT_EQ(changelog4.next_slot(), 5); changelog4.end_of_append_batch(0, 0);
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,198 +1,239 @@
#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
{
/// CREATE TABLE or CREATE DICTIONARY or CREATE VIEW or CREATE TEMPORARY TABLE or CREATE DATABASE query.
void visitCreateQuery(const ASTCreateQuery & create, DDLDependencyVisitor::Data & data)
{
QualifiedTableName to_table{create.to_table_id.database_name, create.to_table_id.table_name};
if (!to_table.table.empty())
{
/// TO target_table (for materialized views)
if (to_table.database.empty())
to_table.database = data.default_database;
data.dependencies.emplace(to_table);
}
TableNamesSet getDependenciesSetFromCreateQuery(ContextPtr global_context, const QualifiedTableName & table, const ASTPtr & ast) QualifiedTableName as_table{create.as_database, create.as_table};
if (!as_table.table.empty())
{
/// AS table_name
if (as_table.database.empty())
as_table.database = data.default_database;
data.dependencies.emplace(as_table);
}
}
/// 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 (!expr.database_and_table_name)
return;
const ASTIdentifier * identifier = dynamic_cast<const ASTIdentifier *>(expr.database_and_table_name.get());
if (!identifier)
return;
auto table_identifier = identifier->createTable();
if (!table_identifier)
return;
QualifiedTableName qualified_name{table_identifier->getDatabaseName(), table_identifier->shortName()};
if (qualified_name.table.empty())
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(qualified_name);
}
/// 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
if (!function.arguments || function.arguments->children.size() <= arg_idx)
return;
QualifiedTableName qualified_name;
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 (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
{
/// Just return because we don't validate AST in this function.
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));
}
/// Extracts a table name with database written in the form 'db_name', 'table_name' (two strings).
void extractDatabaseAndTableNameFromArguments(const ASTFunction & function, DDLDependencyVisitor::Data & data, size_t database_arg_idx, size_t table_arg_idx)
{
/// Just ignore incorrect arguments, proper exception will be thrown later
if (!function.arguments || (function.arguments->children.size() <= database_arg_idx)
|| (function.arguments->children.size() <= table_arg_idx))
return;
const auto * expr_list = function.arguments->as<ASTExpressionList>();
if (!expr_list)
return;
const auto * database_literal = expr_list->children[database_arg_idx]->as<ASTLiteral>();
const auto * table_name_literal = expr_list->children[table_arg_idx]->as<ASTLiteral>();
if (!database_literal || !table_name_literal || (database_literal->value.getType() != Field::Types::String)
|| (table_name_literal->value.getType() != Field::Types::String))
return;
QualifiedTableName qualified_name{database_literal->value.get<String>(), table_name_literal->value.get<String>()};
if (qualified_name.table.empty())
return;
if (qualified_name.database.empty())
qualified_name.database = data.default_database;
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()); assert(global_context == global_context->getGlobalContext());
TableLoadingDependenciesVisitor::Data data; DDLDependencyVisitor::Data data;
data.table_name = table_name;
data.default_database = global_context->getCurrentDatabase(); data.default_database = global_context->getCurrentDatabase();
data.create_query = ast; data.create_query = ast;
data.global_context = global_context; data.global_context = global_context;
TableLoadingDependenciesVisitor visitor{data}; DDLDependencyVisitor::Visitor visitor{data};
visitor.visit(ast); visitor.visit(ast);
data.dependencies.erase(table); data.dependencies.erase(data.table_name);
return data.dependencies; return data.dependencies;
} }
void DDLDependencyVisitor::visit(const ASTPtr & ast, Data & data) void DDLDependencyVisitor::visit(const ASTPtr & ast, Data & data)
{ {
/// Looking for functions in column default expressions and dictionary source definition if (auto * create = ast->as<ASTCreateQuery>())
if (const auto * function = ast->as<ASTFunction>()) {
visit(*function, data); visitCreateQuery(*create, data);
else if (const auto * dict_source = ast->as<ASTFunctionWithKeyValueArguments>()) }
visit(*dict_source, data); else if (auto * dictionary = ast->as<ASTDictionary>())
else if (const auto * storage = ast->as<ASTStorage>()) {
visit(*storage, data); 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 DDLMatcherBase::needChildVisit(const ASTPtr & node, const ASTPtr & child) bool DDLDependencyVisitor::needChildVisit(const ASTPtr &, const ASTPtr &)
{ {
if (node->as<ASTStorage>())
return false;
if (auto * create = node->as<ASTCreateQuery>())
{
if (child.get() == create->select)
return false;
}
return true; 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;
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 DDLDependencyVisitor::visit(const ASTStorage & storage, Data & data)
{
if (!storage.engine)
return;
if (storage.engine->name != "Dictionary")
return;
extractTableNameFromArgument(*storage.engine, data, 0);
}
void DDLDependencyVisitor::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));
}
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

@ -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. for (size_t level = 0; level != tables_to_load.size(); ++level)
/// 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()); startLoadingTables(pool, load_context, tables_to_load[level], level);
logDependencyGraph();
startLoadingIndependentTables(pool, level, load_context);
TableNames new_independent_database_objects;
for (const auto & table_name : metadata.independent_database_objects)
{
auto info_it = metadata.dependencies_info.find(table_name);
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::startLoadingIndependentTables(ThreadPool & pool, size_t level, ContextMutablePtr load_context) void TablesLoader::startLoadingTables(ThreadPool & pool, ContextMutablePtr load_context, const std::vector<StorageID> & tables_to_load, size_t level)
{ {
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

@ -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>())

View File

@ -43,6 +43,16 @@ struct ArrayFirstLastImpl
return array_element; return array_element;
} }
static ColumnPtr createNullableColumn(MutableColumnPtr && column, ColumnUInt8::MutablePtr && null_map)
{
if (auto * nullable_column = typeid_cast<ColumnNullable *>(column.get()))
{
nullable_column->applyNullMap(*null_map);
return std::move(column);
}
return ColumnNullable::create(std::move(column), std::move(null_map));
}
static ColumnPtr execute(const ColumnArray & array, ColumnPtr mapped) static ColumnPtr execute(const ColumnArray & array, ColumnPtr mapped)
{ {
const auto * column_filter = typeid_cast<const ColumnUInt8 *>(&*mapped); const auto * column_filter = typeid_cast<const ColumnUInt8 *>(&*mapped);
@ -94,7 +104,7 @@ struct ArrayFirstLastImpl
} }
if constexpr (element_not_exists_strategy == ArrayFirstLastElementNotExistsStrategy::Null) if constexpr (element_not_exists_strategy == ArrayFirstLastElementNotExistsStrategy::Null)
return ColumnNullable::create(std::move(out), std::move(col_null_map_to)); return createNullableColumn(std::move(out), std::move(col_null_map_to));
return out; return out;
} }
@ -106,7 +116,7 @@ struct ArrayFirstLastImpl
if constexpr (element_not_exists_strategy == ArrayFirstLastElementNotExistsStrategy::Null) if constexpr (element_not_exists_strategy == ArrayFirstLastElementNotExistsStrategy::Null)
{ {
auto col_null_map_to = ColumnUInt8::create(out->size(), true); auto col_null_map_to = ColumnUInt8::create(out->size(), true);
return ColumnNullable::create(std::move(out), std::move(col_null_map_to)); return createNullableColumn(std::move(out), std::move(col_null_map_to));
} }
return out; return out;
@ -172,7 +182,7 @@ struct ArrayFirstLastImpl
} }
if constexpr (element_not_exists_strategy == ArrayFirstLastElementNotExistsStrategy::Null) if constexpr (element_not_exists_strategy == ArrayFirstLastElementNotExistsStrategy::Null)
return ColumnNullable::create(std::move(out), std::move(col_null_map_to)); return createNullableColumn(std::move(out), std::move(col_null_map_to));
return out; return out;
} }

View File

@ -27,14 +27,14 @@ struct TranslateImpl
const std::string & map_to) const std::string & map_to)
{ {
if (map_from.size() != map_to.size()) if (map_from.size() != map_to.size())
throw Exception("Second and trird arguments must be the same length", ErrorCodes::BAD_ARGUMENTS); throw Exception("Second and third arguments must be the same length", ErrorCodes::BAD_ARGUMENTS);
std::iota(map.begin(), map.end(), 0); std::iota(map.begin(), map.end(), 0);
for (size_t i = 0; i < map_from.size(); ++i) for (size_t i = 0; i < map_from.size(); ++i)
{ {
if (!isASCII(map_from[i]) || !isASCII(map_to[i])) if (!isASCII(map_from[i]) || !isASCII(map_to[i]))
throw Exception("Second and trird arguments must be ASCII strings", ErrorCodes::BAD_ARGUMENTS); throw Exception("Second and third arguments must be ASCII strings", ErrorCodes::BAD_ARGUMENTS);
map[map_from[i]] = map_to[i]; map[map_from[i]] = map_to[i];
} }
@ -125,7 +125,7 @@ struct TranslateUTF8Impl
auto map_to_size = UTF8::countCodePoints(reinterpret_cast<const UInt8 *>(map_to.data()), map_to.size()); auto map_to_size = UTF8::countCodePoints(reinterpret_cast<const UInt8 *>(map_to.data()), map_to.size());
if (map_from_size != map_to_size) if (map_from_size != map_to_size)
throw Exception("Second and trird arguments must be the same length", ErrorCodes::BAD_ARGUMENTS); throw Exception("Second and third arguments must be the same length", ErrorCodes::BAD_ARGUMENTS);
std::iota(map_ascii.begin(), map_ascii.end(), 0); std::iota(map_ascii.begin(), map_ascii.end(), 0);

View File

@ -14,6 +14,7 @@
#include <Common/MultiVersion.h> #include <Common/MultiVersion.h>
#include <Common/OpenTelemetryTraceContext.h> #include <Common/OpenTelemetryTraceContext.h>
#include <Common/RemoteHostFilter.h> #include <Common/RemoteHostFilter.h>
#include <Common/ThreadPool.h>
#include <Common/isLocalAddress.h> #include <Common/isLocalAddress.h>
#include <base/types.h> #include <base/types.h>
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h> #include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>

View File

@ -223,6 +223,7 @@ void DatabaseCatalog::shutdownImpl()
return it != elem.map.end(); return it != elem.map.end();
}) == uuid_map.end()); }) == uuid_map.end());
databases.clear(); databases.clear();
referential_dependencies.clear();
view_dependencies.clear(); view_dependencies.clear();
} }
@ -473,13 +474,8 @@ void DatabaseCatalog::updateDatabaseName(const String & old_name, const String &
for (const auto & table_name : tables_in_database) for (const auto & table_name : tables_in_database)
{ {
QualifiedTableName new_table_name{new_name, table_name}; auto dependencies = referential_dependencies.removeDependencies(StorageID{old_name, table_name}, /* remove_isolated_tables= */ true);
auto dependencies = tryRemoveLoadingDependenciesUnlocked(QualifiedTableName{old_name, table_name}, /* check_dependencies */ false); referential_dependencies.addDependencies(StorageID{new_name, table_name}, dependencies);
DependenciesInfos new_info;
for (const auto & dependency : dependencies)
new_info[dependency].dependent_database_objects.insert(new_table_name);
new_info[new_table_name].dependencies = std::move(dependencies);
mergeDependenciesGraphs(loading_dependencies, new_info);
} }
} }
@ -648,7 +644,10 @@ bool DatabaseCatalog::hasUUIDMapping(const UUID & uuid)
std::unique_ptr<DatabaseCatalog> DatabaseCatalog::database_catalog; std::unique_ptr<DatabaseCatalog> DatabaseCatalog::database_catalog;
DatabaseCatalog::DatabaseCatalog(ContextMutablePtr global_context_) DatabaseCatalog::DatabaseCatalog(ContextMutablePtr global_context_)
: WithMutableContext(global_context_), log(&Poco::Logger::get("DatabaseCatalog")) : WithMutableContext(global_context_)
, referential_dependencies{"ReferentialDeps"}
, view_dependencies{"ViewDeps"}
, log(&Poco::Logger::get("DatabaseCatalog"))
{ {
} }
@ -692,39 +691,33 @@ DatabasePtr DatabaseCatalog::getDatabase(const String & database_name, ContextPt
return getDatabase(resolved_database); return getDatabase(resolved_database);
} }
void DatabaseCatalog::addDependency(const StorageID & from, const StorageID & where) void DatabaseCatalog::addViewDependency(const StorageID & source_table_id, const StorageID & view_id)
{ {
std::lock_guard lock{databases_mutex}; std::lock_guard lock{databases_mutex};
// FIXME when loading metadata storage may not know UUIDs of it's dependencies, because they are not loaded yet, view_dependencies.addDependency(source_table_id, view_id);
// so UUID of `from` is not used here. (same for remove, get and update)
view_dependencies[{from.getDatabaseName(), from.getTableName()}].insert(where);
} }
void DatabaseCatalog::removeDependency(const StorageID & from, const StorageID & where) void DatabaseCatalog::removeViewDependency(const StorageID & source_table_id, const StorageID & view_id)
{ {
std::lock_guard lock{databases_mutex}; std::lock_guard lock{databases_mutex};
view_dependencies[{from.getDatabaseName(), from.getTableName()}].erase(where); view_dependencies.removeDependency(source_table_id, view_id, /* remove_isolated_tables= */ true);
} }
Dependencies DatabaseCatalog::getDependencies(const StorageID & from) const std::vector<StorageID> DatabaseCatalog::getDependentViews(const StorageID & source_table_id) const
{ {
std::lock_guard lock{databases_mutex}; std::lock_guard lock{databases_mutex};
auto iter = view_dependencies.find({from.getDatabaseName(), from.getTableName()}); return view_dependencies.getDependencies(source_table_id);
if (iter == view_dependencies.end())
return {};
return Dependencies(iter->second.begin(), iter->second.end());
} }
void void DatabaseCatalog::updateViewDependency(const StorageID & old_source_table_id, const StorageID & old_view_id,
DatabaseCatalog::updateDependency(const StorageID & old_from, const StorageID & old_where, const StorageID & new_from, const StorageID & new_source_table_id, const StorageID & new_view_id)
const StorageID & new_where)
{ {
std::lock_guard lock{databases_mutex}; std::lock_guard lock{databases_mutex};
if (!old_from.empty()) if (!old_source_table_id.empty())
view_dependencies[{old_from.getDatabaseName(), old_from.getTableName()}].erase(old_where); view_dependencies.removeDependency(old_source_table_id, old_view_id, /* remove_isolated_tables= */ true);
if (!new_from.empty()) if (!new_source_table_id.empty())
view_dependencies[{new_from.getDatabaseName(), new_from.getTableName()}].insert(new_where); view_dependencies.addDependency(new_source_table_id, new_view_id);
} }
DDLGuardPtr DatabaseCatalog::getDDLGuard(const String & database, const String & table) DDLGuardPtr DatabaseCatalog::getDDLGuard(const String & database, const String & table)
@ -869,6 +862,8 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr
{ {
chassert(hasUUIDMapping(table_id.uuid)); chassert(hasUUIDMapping(table_id.uuid));
drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now());
/// Do not postpone removal of in-memory tables
ignore_delay = ignore_delay || !table->storesDataOnDisk();
table->is_dropped = true; table->is_dropped = true;
} }
else else
@ -1048,121 +1043,79 @@ void DatabaseCatalog::waitTableFinallyDropped(const UUID & uuid)
}); });
} }
void DatabaseCatalog::addLoadingDependencies(const QualifiedTableName & table, TableNamesSet && dependencies) void DatabaseCatalog::addDependencies(const StorageID & table_id, const std::vector<StorageID> & dependencies)
{
DependenciesInfos new_info;
for (const auto & dependency : dependencies)
new_info[dependency].dependent_database_objects.insert(table);
new_info[table].dependencies = std::move(dependencies);
addLoadingDependencies(new_info);
}
void DatabaseCatalog::addLoadingDependencies(const DependenciesInfos & new_infos)
{ {
std::lock_guard lock{databases_mutex}; std::lock_guard lock{databases_mutex};
mergeDependenciesGraphs(loading_dependencies, new_infos); referential_dependencies.addDependencies(table_id, dependencies);
} }
DependenciesInfo DatabaseCatalog::getLoadingDependenciesInfo(const StorageID & table_id) const void DatabaseCatalog::addDependencies(const QualifiedTableName & table_name, const TableNamesSet & dependencies)
{ {
std::lock_guard lock{databases_mutex}; std::lock_guard lock{databases_mutex};
auto it = loading_dependencies.find(table_id.getQualifiedName()); referential_dependencies.addDependencies(table_name, dependencies);
if (it == loading_dependencies.end())
return {};
return it->second;
} }
TableNamesSet DatabaseCatalog::tryRemoveLoadingDependencies(const StorageID & table_id, bool check_dependencies, bool is_drop_database) void DatabaseCatalog::addDependencies(const TablesDependencyGraph & extra_graph)
{ {
QualifiedTableName removing_table = table_id.getQualifiedName();
std::lock_guard lock{databases_mutex}; std::lock_guard lock{databases_mutex};
return tryRemoveLoadingDependenciesUnlocked(removing_table, check_dependencies, is_drop_database); referential_dependencies.mergeWith(extra_graph);
} }
TableNamesSet DatabaseCatalog::tryRemoveLoadingDependenciesUnlocked(const QualifiedTableName & removing_table, bool check_dependencies, bool is_drop_database) std::vector<StorageID> DatabaseCatalog::getDependencies(const StorageID & table_id) const
{ {
auto it = loading_dependencies.find(removing_table); std::lock_guard lock{databases_mutex};
if (it == loading_dependencies.end()) return referential_dependencies.getDependencies(table_id);
return {}; }
TableNamesSet & dependent = it->second.dependent_database_objects; std::vector<StorageID> DatabaseCatalog::getDependents(const StorageID & table_id) const
if (!dependent.empty()) {
{ std::lock_guard lock{databases_mutex};
if (check_dependencies) return referential_dependencies.getDependents(table_id);
checkTableCanBeRemovedOrRenamedImpl(dependent, removing_table, is_drop_database); }
for (const auto & table : dependent) std::vector<StorageID> DatabaseCatalog::removeDependencies(const StorageID & table_id, bool check_dependencies, bool is_drop_database)
{ {
[[maybe_unused]] bool removed = loading_dependencies[table].dependencies.erase(removing_table); std::lock_guard lock{databases_mutex};
assert(removed); if (check_dependencies)
} checkTableCanBeRemovedOrRenamedUnlocked(table_id, is_drop_database);
dependent.clear(); return referential_dependencies.removeDependencies(table_id, /* remove_isolated_tables= */ true);
}
TableNamesSet dependencies = it->second.dependencies;
for (const auto & table : dependencies)
{
[[maybe_unused]] bool removed = loading_dependencies[table].dependent_database_objects.erase(removing_table);
assert(removed);
}
loading_dependencies.erase(it);
return dependencies;
} }
void DatabaseCatalog::checkTableCanBeRemovedOrRenamed(const StorageID & table_id, bool is_drop_database) const void DatabaseCatalog::checkTableCanBeRemovedOrRenamed(const StorageID & table_id, bool is_drop_database) const
{ {
QualifiedTableName removing_table = table_id.getQualifiedName();
std::lock_guard lock{databases_mutex}; std::lock_guard lock{databases_mutex};
auto it = loading_dependencies.find(removing_table); return checkTableCanBeRemovedOrRenamedUnlocked(table_id, is_drop_database);
if (it == loading_dependencies.end())
return;
const TableNamesSet & dependent = it->second.dependent_database_objects;
checkTableCanBeRemovedOrRenamedImpl(dependent, removing_table, is_drop_database);
} }
void DatabaseCatalog::checkTableCanBeRemovedOrRenamedImpl(const TableNamesSet & dependent, const QualifiedTableName & removing_table, bool is_drop_database) void DatabaseCatalog::checkTableCanBeRemovedOrRenamedUnlocked(const StorageID & removing_table, bool is_drop_database) const
{ {
const auto & dependents = referential_dependencies.getDependents(removing_table);
if (!is_drop_database) if (!is_drop_database)
{ {
if (!dependent.empty()) if (!dependents.empty())
throw Exception(ErrorCodes::HAVE_DEPENDENT_OBJECTS, "Cannot drop or rename {}, because some tables depend on it: {}", throw Exception(ErrorCodes::HAVE_DEPENDENT_OBJECTS, "Cannot drop or rename {}, because some tables depend on it: {}",
removing_table, fmt::join(dependent, ", ")); removing_table, fmt::join(dependents, ", "));
return;
} }
/// For DROP DATABASE we should ignore dependent tables from the same database. /// For DROP DATABASE we should ignore dependent tables from the same database.
/// TODO unload tables in reverse topological order and remove this code /// TODO unload tables in reverse topological order and remove this code
TableNames from_other_databases; std::vector<StorageID> from_other_databases;
for (const auto & table : dependent) for (const auto & dependent : dependents)
if (table.database != removing_table.database) if (dependent.database_name != removing_table.database_name)
from_other_databases.push_back(table); from_other_databases.push_back(dependent);
if (!from_other_databases.empty()) if (!from_other_databases.empty())
throw Exception(ErrorCodes::HAVE_DEPENDENT_OBJECTS, "Cannot drop or rename {}, because some tables depend on it: {}", throw Exception(ErrorCodes::HAVE_DEPENDENT_OBJECTS, "Cannot drop or rename {}, because some tables depend on it: {}",
removing_table, fmt::join(from_other_databases, ", ")); removing_table, fmt::join(from_other_databases, ", "));
} }
void DatabaseCatalog::updateLoadingDependencies(const StorageID & table_id, TableNamesSet && new_dependencies) void DatabaseCatalog::updateDependencies(const StorageID & table_id, const TableNamesSet & new_dependencies)
{ {
if (new_dependencies.empty())
return;
QualifiedTableName table_name = table_id.getQualifiedName();
std::lock_guard lock{databases_mutex}; std::lock_guard lock{databases_mutex};
auto it = loading_dependencies.find(table_name); referential_dependencies.removeDependencies(table_id, /* remove_isolated_tables= */ true);
if (it == loading_dependencies.end()) referential_dependencies.addDependencies(table_id, new_dependencies);
it = loading_dependencies.emplace(table_name, DependenciesInfo{}).first;
auto & old_dependencies = it->second.dependencies;
for (const auto & dependency : old_dependencies)
if (!new_dependencies.contains(dependency))
loading_dependencies[dependency].dependent_database_objects.erase(table_name);
for (const auto & dependency : new_dependencies)
if (!old_dependencies.contains(dependency))
loading_dependencies[dependency].dependent_database_objects.insert(table_name);
old_dependencies = std::move(new_dependencies);
} }
void DatabaseCatalog::cleanupStoreDirectoryTask() void DatabaseCatalog::cleanupStoreDirectoryTask()

View File

@ -3,7 +3,7 @@
#include <Core/UUID.h> #include <Core/UUID.h>
#include <Interpreters/Context_fwd.h> #include <Interpreters/Context_fwd.h>
#include <Interpreters/StorageID.h> #include <Interpreters/StorageID.h>
#include <Databases/TablesLoader.h> #include <Databases/TablesDependencyGraph.h>
#include <Parsers/IAST_fwd.h> #include <Parsers/IAST_fwd.h>
#include <Storages/IStorage_fwd.h> #include <Storages/IStorage_fwd.h>
@ -37,11 +37,7 @@ using DatabasePtr = std::shared_ptr<IDatabase>;
using DatabaseAndTable = std::pair<DatabasePtr, StoragePtr>; using DatabaseAndTable = std::pair<DatabasePtr, StoragePtr>;
using Databases = std::map<String, std::shared_ptr<IDatabase>>; using Databases = std::map<String, std::shared_ptr<IDatabase>>;
using DiskPtr = std::shared_ptr<IDisk>; using DiskPtr = std::shared_ptr<IDisk>;
using TableNamesSet = std::unordered_set<QualifiedTableName>;
/// Table -> set of table-views that make SELECT from it.
using ViewDependencies = std::map<StorageID, std::set<StorageID>>;
using Dependencies = std::vector<StorageID>;
/// Allows executing DDL query only in one thread. /// Allows executing DDL query only in one thread.
/// Puts an element into the map, locks tables's mutex, counts how much threads run parallel query on the table, /// Puts an element into the map, locks tables's mutex, counts how much threads run parallel query on the table,
@ -188,12 +184,11 @@ public:
/// Four views (tables, views, columns, schemata) in the "information_schema" database are predefined too. /// Four views (tables, views, columns, schemata) in the "information_schema" database are predefined too.
bool isPredefinedTable(const StorageID & table_id) const; bool isPredefinedTable(const StorageID & table_id) const;
void addDependency(const StorageID & from, const StorageID & where); /// View dependencies between a source table and its view.
void removeDependency(const StorageID & from, const StorageID & where); void addViewDependency(const StorageID & source_table_id, const StorageID & view_id);
Dependencies getDependencies(const StorageID & from) const; void removeViewDependency(const StorageID & source_table_id, const StorageID & view_id);
std::vector<StorageID> getDependentViews(const StorageID & source_table_id) const;
/// For Materialized and Live View void updateViewDependency(const StorageID & old_source_table_id, const StorageID & old_view_id, const StorageID & new_source_table_id, const StorageID & new_view_id);
void updateDependency(const StorageID & old_from, const StorageID & old_where,const StorageID & new_from, const StorageID & new_where);
/// If table has UUID, addUUIDMapping(...) must be called when table attached to some database /// If table has UUID, addUUIDMapping(...) must be called when table attached to some database
/// removeUUIDMapping(...) must be called when it detached, /// removeUUIDMapping(...) must be called when it detached,
@ -223,16 +218,20 @@ public:
void waitTableFinallyDropped(const UUID & uuid); void waitTableFinallyDropped(const UUID & uuid);
void addLoadingDependencies(const QualifiedTableName & table, TableNamesSet && dependencies); /// Referential dependencies between tables: table "A" depends on table "B"
void addLoadingDependencies(const DependenciesInfos & new_infos); /// if "B" is referenced in the definition of "A".
DependenciesInfo getLoadingDependenciesInfo(const StorageID & table_id) const; void addDependencies(const StorageID & table_id, const std::vector<StorageID> & dependencies);
void addDependencies(const QualifiedTableName & table_name, const TableNamesSet & dependencies);
void addDependencies(const TablesDependencyGraph & extra_graph);
std::vector<StorageID> removeDependencies(const StorageID & table_id, bool check_dependencies, bool is_drop_database = false);
std::vector<StorageID> getDependencies(const StorageID & table_id) const;
std::vector<StorageID> getDependents(const StorageID & table_id) const;
void updateDependencies(const StorageID & table_id, const TableNamesSet & new_dependencies);
TableNamesSet tryRemoveLoadingDependencies(const StorageID & table_id, bool check_dependencies, bool is_drop_database = false);
TableNamesSet tryRemoveLoadingDependenciesUnlocked(const QualifiedTableName & removing_table, bool check_dependencies, bool is_drop_database = false) TSA_REQUIRES(databases_mutex);
void checkTableCanBeRemovedOrRenamed(const StorageID & table_id, bool is_drop_database = false) const; void checkTableCanBeRemovedOrRenamed(const StorageID & table_id, bool is_drop_database = false) const;
void updateLoadingDependencies(const StorageID & table_id, TableNamesSet && new_dependencies);
private: private:
// The global instance of database catalog. unique_ptr is to allow // The global instance of database catalog. unique_ptr is to allow
// deferred initialization. Thought I'd use std::optional, but I can't // deferred initialization. Thought I'd use std::optional, but I can't
@ -245,7 +244,7 @@ private:
void shutdownImpl(); void shutdownImpl();
static void checkTableCanBeRemovedOrRenamedImpl(const TableNamesSet & dependent, const QualifiedTableName & removing_table, bool is_drop_database); void checkTableCanBeRemovedOrRenamedUnlocked(const StorageID & removing_table, bool is_drop_database) const TSA_REQUIRES(databases_mutex);
struct UUIDToStorageMapPart struct UUIDToStorageMapPart
{ {
@ -281,12 +280,15 @@ private:
mutable std::mutex databases_mutex; mutable std::mutex databases_mutex;
ViewDependencies view_dependencies TSA_GUARDED_BY(databases_mutex);
Databases databases TSA_GUARDED_BY(databases_mutex); Databases databases TSA_GUARDED_BY(databases_mutex);
UUIDToStorageMap uuid_map; UUIDToStorageMap uuid_map;
DependenciesInfos loading_dependencies TSA_GUARDED_BY(databases_mutex); /// Referential dependencies between tables: table "A" depends on table "B"
/// if the table "B" is referenced in the definition of the table "A".
TablesDependencyGraph referential_dependencies TSA_GUARDED_BY(databases_mutex);
/// View dependencies between a source table and its view.
TablesDependencyGraph view_dependencies TSA_GUARDED_BY(databases_mutex);
Poco::Logger * log; Poco::Logger * log;

View File

@ -58,6 +58,7 @@
#include <Databases/DatabaseOnDisk.h> #include <Databases/DatabaseOnDisk.h>
#include <Databases/TablesLoader.h> #include <Databases/TablesLoader.h>
#include <Databases/DDLDependencyVisitor.h> #include <Databases/DDLDependencyVisitor.h>
#include <Databases/NormalizeAndEvaluateConstantsVisitor.h>
#include <Compression/CompressionFactory.h> #include <Compression/CompressionFactory.h>
@ -1234,9 +1235,9 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
/// If table has dependencies - add them to the graph /// If table has dependencies - add them to the graph
QualifiedTableName qualified_name{database_name, create.getTable()}; QualifiedTableName qualified_name{database_name, create.getTable()};
TableNamesSet loading_dependencies = getDependenciesSetFromCreateQuery(getContext()->getGlobalContext(), qualified_name, query_ptr); TableNamesSet dependencies = getLoadingDependenciesFromCreateQuery(getContext()->getGlobalContext(), qualified_name, query_ptr);
if (!loading_dependencies.empty()) if (!dependencies.empty())
DatabaseCatalog::instance().addLoadingDependencies(qualified_name, std::move(loading_dependencies)); DatabaseCatalog::instance().addDependencies(qualified_name, dependencies);
return fillTableIfNeeded(create); return fillTableIfNeeded(create);
} }

View File

@ -188,8 +188,8 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue
if (query.permanently) if (query.permanently)
{ {
/// Server may fail to restart of DETACH PERMANENTLY if table has dependent ones /// Server may fail to restart of DETACH PERMANENTLY if table has dependent ones
DatabaseCatalog::instance().tryRemoveLoadingDependencies(table_id, getContext()->getSettingsRef().check_table_dependencies, DatabaseCatalog::instance().removeDependencies(table_id, getContext()->getSettingsRef().check_table_dependencies,
is_drop_or_detach_database); is_drop_or_detach_database);
/// Drop table from memory, don't touch data, metadata file renamed and will be skipped during server restart /// Drop table from memory, don't touch data, metadata file renamed and will be skipped during server restart
database->detachTablePermanently(context_, table_id.table_name); database->detachTablePermanently(context_, table_id.table_name);
} }
@ -243,8 +243,8 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue
if (database->getUUID() == UUIDHelpers::Nil) if (database->getUUID() == UUIDHelpers::Nil)
table_lock = table->lockExclusively(context_->getCurrentQueryId(), context_->getSettingsRef().lock_acquire_timeout); table_lock = table->lockExclusively(context_->getCurrentQueryId(), context_->getSettingsRef().lock_acquire_timeout);
DatabaseCatalog::instance().tryRemoveLoadingDependencies(table_id, getContext()->getSettingsRef().check_table_dependencies, DatabaseCatalog::instance().removeDependencies(table_id, getContext()->getSettingsRef().check_table_dependencies,
is_drop_or_detach_database); is_drop_or_detach_database);
database->dropTable(context_, table_id.table_name, query.sync); database->dropTable(context_, table_id.table_name, query.sync);
/// We have to drop mmapio cache when dropping table from Ordinary database /// We have to drop mmapio cache when dropping table from Ordinary database

View File

@ -124,10 +124,10 @@ BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, c
} }
else else
{ {
TableNamesSet dependencies; std::vector<StorageID> dependencies;
if (!exchange_tables) if (!exchange_tables)
dependencies = database_catalog.tryRemoveLoadingDependencies(StorageID(elem.from_database_name, elem.from_table_name), dependencies = database_catalog.removeDependencies(StorageID(elem.from_database_name, elem.from_table_name),
getContext()->getSettingsRef().check_table_dependencies); getContext()->getSettingsRef().check_table_dependencies);
database->renameTable( database->renameTable(
getContext(), getContext(),
@ -138,7 +138,7 @@ BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, c
rename.dictionary); rename.dictionary);
if (!dependencies.empty()) if (!dependencies.empty())
DatabaseCatalog::instance().addLoadingDependencies(QualifiedTableName{elem.to_database_name, elem.to_table_name}, std::move(dependencies)); DatabaseCatalog::instance().addDependencies(StorageID(elem.to_database_name, elem.to_table_name), dependencies);
} }
} }

View File

@ -230,6 +230,7 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as
/// Set up memory profiling /// Set up memory profiling
thread_group->memory_tracker.setProfilerStep(settings.memory_profiler_step); thread_group->memory_tracker.setProfilerStep(settings.memory_profiler_step);
thread_group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability); thread_group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability);
thread_group->performance_counters.setTraceProfileEvents(settings.trace_profile_events);
} }
thread_group->memory_tracker.setDescription("(for query)"); thread_group->memory_tracker.setDescription("(for query)");

View File

@ -64,21 +64,8 @@ String StorageID::getNameForLogs() const
+ (hasUUID() ? " (" + toString(uuid) + ")" : ""); + (hasUUID() ? " (" + toString(uuid) + ")" : "");
} }
bool StorageID::operator<(const StorageID & rhs) const /// NOTE: This implementation doesn't allow to implement a good "operator <".
{ /// Because "a != b" must be equivalent to "(a < b) || (b < a)", and we can't make "operator <" to meet that.
assertNotEmpty();
/// It's needed for ViewDependencies
if (!hasUUID() && !rhs.hasUUID())
/// If both IDs don't have UUID, compare them like pair of strings
return std::tie(database_name, table_name) < std::tie(rhs.database_name, rhs.table_name);
else if (hasUUID() && rhs.hasUUID())
/// If both IDs have UUID, compare UUIDs and ignore database and table name
return uuid < rhs.uuid;
else
/// All IDs without UUID are less, then all IDs with UUID
return !hasUUID();
}
bool StorageID::operator==(const StorageID & rhs) const bool StorageID::operator==(const StorageID & rhs) const
{ {
assertNotEmpty(); assertNotEmpty();

View File

@ -45,6 +45,8 @@ struct StorageID
StorageID(const ASTTableIdentifier & table_identifier_node); /// NOLINT StorageID(const ASTTableIdentifier & table_identifier_node); /// NOLINT
StorageID(const ASTPtr & node); /// NOLINT StorageID(const ASTPtr & node); /// NOLINT
explicit StorageID(const QualifiedTableName & qualified_name) : StorageID(qualified_name.database, qualified_name.table) { }
String getDatabaseName() const; String getDatabaseName() const;
String getTableName() const; String getTableName() const;
@ -71,7 +73,6 @@ struct StorageID
bool hasDatabase() const { return !database_name.empty(); } bool hasDatabase() const { return !database_name.empty(); }
bool operator<(const StorageID & rhs) const;
bool operator==(const StorageID & rhs) const; bool operator==(const StorageID & rhs) const;
void assertNotEmpty() const void assertNotEmpty() const
@ -97,8 +98,47 @@ struct StorageID
/// Get short, but unique, name. /// Get short, but unique, name.
String getShortName() const; String getShortName() const;
/// Calculates hash using only the database and table name of a StorageID.
struct DatabaseAndTableNameHash
{
size_t operator()(const StorageID & storage_id) const
{
SipHash hash_state;
hash_state.update(storage_id.database_name.data(), storage_id.database_name.size());
hash_state.update(storage_id.table_name.data(), storage_id.table_name.size());
return hash_state.get64();
}
};
/// Checks if the database and table name of two StorageIDs are equal.
struct DatabaseAndTableNameEqual
{
bool operator()(const StorageID & left, const StorageID & right) const
{
return (left.database_name == right.database_name) && (left.table_name == right.table_name);
}
};
private: private:
StorageID() = default; StorageID() = default;
}; };
} }
namespace fmt
{
template <>
struct formatter<DB::StorageID>
{
static constexpr auto parse(format_parse_context & ctx)
{
return ctx.begin();
}
template <typename FormatContext>
auto format(const DB::StorageID & storage_id, FormatContext & ctx)
{
return format_to(ctx.out(), "{}", storage_id.getNameForLogs());
}
};
}

View File

@ -7,6 +7,7 @@
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <Interpreters/TraceLog.h> #include <Interpreters/TraceLog.h>
#include <Poco/Logger.h> #include <Poco/Logger.h>
#include <Common/ProfileEvents.h>
#include <Common/setThreadName.h> #include <Common/setThreadName.h>
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
@ -96,6 +97,12 @@ void TraceCollector::run()
Int64 size; Int64 size;
readPODBinary(size, in); readPODBinary(size, in);
ProfileEvents::Event event;
readPODBinary(event, in);
ProfileEvents::Count increment;
readPODBinary(increment, in);
if (trace_log) if (trace_log)
{ {
// time and time_in_microseconds are both being constructed from the same timespec so that the // time and time_in_microseconds are both being constructed from the same timespec so that the
@ -105,7 +112,7 @@ void TraceCollector::run()
UInt64 time = static_cast<UInt64>(ts.tv_sec * 1000000000LL + ts.tv_nsec); UInt64 time = static_cast<UInt64>(ts.tv_sec * 1000000000LL + ts.tv_nsec);
UInt64 time_in_microseconds = static_cast<UInt64>((ts.tv_sec * 1000000LL) + (ts.tv_nsec / 1000)); UInt64 time_in_microseconds = static_cast<UInt64>((ts.tv_sec * 1000000LL) + (ts.tv_nsec / 1000));
TraceLogElement element{time_t(time / 1000000000), time_in_microseconds, time, trace_type, thread_id, query_id, trace, size}; TraceLogElement element{time_t(time / 1000000000), time_in_microseconds, time, trace_type, thread_id, query_id, trace, size, event, increment};
trace_log->add(element); trace_log->add(element);
} }
} }

View File

@ -1,7 +1,5 @@
#pragma once #pragma once
#include <Common/ThreadPool.h> #include <Common/ThreadPool.h>
#include <Common/TraceSender.h>
class StackTrace; class StackTrace;
@ -21,11 +19,6 @@ public:
explicit TraceCollector(std::shared_ptr<TraceLog> trace_log_); explicit TraceCollector(std::shared_ptr<TraceLog> trace_log_);
~TraceCollector(); ~TraceCollector();
static inline void collect(TraceType trace_type, const StackTrace & stack_trace, Int64 size)
{
return TraceSender::send(trace_type, stack_trace, size);
}
private: private:
std::shared_ptr<TraceLog> trace_log; std::shared_ptr<TraceLog> trace_log;
ThreadFromGlobalPool thread; ThreadFromGlobalPool thread;

View File

@ -5,6 +5,7 @@
#include <DataTypes/DataTypeDate.h> #include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h> #include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h> #include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Common/ClickHouseRevision.h> #include <Common/ClickHouseRevision.h>
@ -20,6 +21,7 @@ const TraceDataType::Values TraceLogElement::trace_values =
{"Memory", static_cast<UInt8>(TraceType::Memory)}, {"Memory", static_cast<UInt8>(TraceType::Memory)},
{"MemorySample", static_cast<UInt8>(TraceType::MemorySample)}, {"MemorySample", static_cast<UInt8>(TraceType::MemorySample)},
{"MemoryPeak", static_cast<UInt8>(TraceType::MemoryPeak)}, {"MemoryPeak", static_cast<UInt8>(TraceType::MemoryPeak)},
{"ProfileEvent", static_cast<UInt8>(TraceType::ProfileEvent)},
}; };
NamesAndTypesList TraceLogElement::getNamesAndTypes() NamesAndTypesList TraceLogElement::getNamesAndTypes()
@ -36,6 +38,8 @@ NamesAndTypesList TraceLogElement::getNamesAndTypes()
{"query_id", std::make_shared<DataTypeString>()}, {"query_id", std::make_shared<DataTypeString>()},
{"trace", std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>())}, {"trace", std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>())},
{"size", std::make_shared<DataTypeInt64>()}, {"size", std::make_shared<DataTypeInt64>()},
{"event", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
{"increment", std::make_shared<DataTypeInt64>()},
}; };
} }
@ -53,6 +57,13 @@ void TraceLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insertData(query_id.data(), query_id.size()); columns[i++]->insertData(query_id.data(), query_id.size());
columns[i++]->insert(trace); columns[i++]->insert(trace);
columns[i++]->insert(size); columns[i++]->insert(size);
String event_name;
if (event != ProfileEvents::end())
event_name = ProfileEvents::getName(event);
columns[i++]->insert(event_name);
columns[i++]->insert(increment);
} }
} }

View File

@ -3,8 +3,9 @@
#include <DataTypes/DataTypeEnum.h> #include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <Interpreters/SystemLog.h> #include <Interpreters/SystemLog.h>
#include <Interpreters/TraceCollector.h>
#include <Common/QueryProfiler.h> #include <Common/QueryProfiler.h>
#include <Common/ProfileEvents.h>
#include <Common/TraceSender.h>
#include <Core/NamesAndTypes.h> #include <Core/NamesAndTypes.h>
#include <Core/NamesAndAliases.h> #include <Core/NamesAndAliases.h>
@ -26,7 +27,12 @@ struct TraceLogElement
UInt64 thread_id{}; UInt64 thread_id{};
String query_id{}; String query_id{};
Array trace{}; Array trace{};
Int64 size{}; /// Allocation size in bytes for TraceType::Memory /// Allocation size in bytes for TraceType::Memory.
Int64 size{};
/// ProfileEvent for TraceType::ProfileEvent.
ProfileEvents::Event event{ProfileEvents::end()};
/// Increment of profile event for TraceType::ProfileEvent.
ProfileEvents::Count increment{};
static std::string name() { return "TraceLog"; } static std::string name() { return "TraceLog"; }
static NamesAndTypesList getNamesAndTypes(); static NamesAndTypesList getNamesAndTypes();

View File

@ -1,7 +1,6 @@
#pragma once #pragma once
#include <Interpreters/Context_fwd.h> #include <Interpreters/Context_fwd.h>
#include <Databases/TablesLoader.h>
namespace DB namespace DB

View File

@ -509,7 +509,7 @@ bool ASTAlterQuery::isOneCommandTypeOnly(const ASTAlterCommand::Type & type) con
bool ASTAlterQuery::isSettingsAlter() const bool ASTAlterQuery::isSettingsAlter() const
{ {
return isOneCommandTypeOnly(ASTAlterCommand::MODIFY_SETTING); return isOneCommandTypeOnly(ASTAlterCommand::MODIFY_SETTING) || isOneCommandTypeOnly(ASTAlterCommand::RESET_SETTING);
} }
bool ASTAlterQuery::isFreezeAlter() const bool ASTAlterQuery::isFreezeAlter() const

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