Merge branch 'master' into ADQM-623

This commit is contained in:
Roman Vasin 2022-12-04 20:21:27 +03:00 committed by GitHub
commit bcb65eb420
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
170 changed files with 2919 additions and 2171 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

@ -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.
:::
## 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}
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

@ -1807,6 +1807,41 @@ See also:
- 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}
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.
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.
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.
@ -29,6 +30,8 @@ Columns:
- `CPU` represents collecting stack traces by CPU time.
- `Memory` represents collecting allocations and deallocations when memory allocation exceeds the subsequent watermark.
- `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.
@ -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.
- `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**
``` 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 p-value. [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-low. [Float64](../../../sql-reference/data-types/float.md).
- calculated confidence-interval-high. [Float64](../../../sql-reference/data-types/float.md).
**Example**

View File

@ -1865,6 +1865,17 @@ Next, specify the path to `libcatboostmodel.<so|dylib>` in the clickhouse config
</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
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

@ -207,7 +207,7 @@ void LocalServer::tryInitPath()
global_context->setPath(path);
global_context->setTemporaryStoragePath(path + "tmp/", 0);
global_context->setTemporaryStorage(path + "tmp", "", 0);
global_context->setFlagsPath(path + "flags");
global_context->setUserFilesPath(""); // user's files are everywhere

View File

@ -203,6 +203,46 @@ int mainEntryClickHouseServer(int argc, char ** argv)
namespace
{
void setupTmpPath(Poco::Logger * log, const std::string & path)
try
{
LOG_DEBUG(log, "Setting up {} to store temporary data in it", path);
fs::create_directories(path);
/// Clearing old temporary files.
fs::directory_iterator dir_end;
size_t unknown_files = 0;
for (fs::directory_iterator it(path); it != dir_end; ++it)
{
if (it->is_regular_file() && startsWith(it->path().filename(), "tmp"))
{
LOG_DEBUG(log, "Removing old temporary file {}", it->path().string());
fs::remove(it->path());
}
else
{
unknown_files++;
if (unknown_files < 100)
LOG_DEBUG(log, "Found unknown {} {} in temporary path",
it->is_regular_file() ? "file" : (it->is_directory() ? "directory" : "element"),
it->path().string());
}
}
if (unknown_files)
LOG_DEBUG(log, "Found {} unknown files in temporary path", unknown_files);
}
catch (...)
{
DB::tryLogCurrentException(
log,
fmt::format(
"Caught exception while setup temporary path: {}. It is ok to skip this exception as cleaning old temporary files is not "
"necessary",
path));
}
size_t waitServersToFinish(std::vector<DB::ProtocolServerAdapter> & servers, size_t seconds_to_wait)
{
const size_t sleep_max_ms = 1000 * seconds_to_wait;
@ -997,21 +1037,13 @@ try
LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::instance().getTimeZone());
/// Storage with temporary data for processing of heavy queries.
if (auto temporary_policy = config().getString("tmp_policy", ""); !temporary_policy.empty())
{
size_t max_size = config().getUInt64("max_temporary_data_on_disk_size", 0);
global_context->setTemporaryStoragePolicy(temporary_policy, max_size);
}
else if (auto temporary_cache = config().getString("tmp_cache", ""); !temporary_cache.empty())
{
size_t max_size = config().getUInt64("max_temporary_data_on_disk_size", 0);
global_context->setTemporaryStorageInCache(temporary_cache, max_size);
}
else
{
std::string temporary_path = config().getString("tmp_path", path / "tmp/");
std::string temporary_policy = config().getString("tmp_policy", "");
size_t max_size = config().getUInt64("max_temporary_data_on_disk_size", 0);
global_context->setTemporaryStoragePath(temporary_path, max_size);
const VolumePtr & volume = global_context->setTemporaryStorage(temporary_path, temporary_policy, max_size);
for (const DiskPtr & disk : volume->getDisks())
setupTmpPath(log, disk->getPath());
}
/** Directory with 'flags': files indicating temporary settings for the server set by system administrator.
@ -1410,7 +1442,7 @@ try
}
catch (...)
{
tryLogCurrentException(log, "Caught exception while setting up access control.");
tryLogCurrentException(log);
throw;
}

View File

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

View File

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

View File

@ -296,19 +296,19 @@ public:
{
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;
}
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;
}
}
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)
serialize(col, elem.second);
}
@ -328,12 +328,12 @@ public:
{
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;
}
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;
}
}
@ -341,7 +341,7 @@ public:
for (size_t i = 0; i < size; ++i)
{
Field key;
keys_serialization->deserializeBinary(key, buf);
keys_serialization->deserializeBinary(key, buf, {});
Array values;
values.resize(values_types.size());

View File

@ -96,6 +96,7 @@ RestorerFromBackup::RestorerFromBackup(
, 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))
, 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.
setStage(Stage::CREATING_TABLES);
removeUnresolvedDependencies();
createTables();
/// 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];
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.dependencies = getDependenciesSetFromCreateQuery(context->getGlobalContext(), table_name, create_table_query);
res_table_info.has_data = backup->hasFiles(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 (!res_table_info.partitions)
@ -622,22 +625,63 @@ 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()
{
while (true)
{
/// We need to create tables considering their dependencies.
auto tables_to_create = findTablesWithoutDependencies();
if (tables_to_create.empty())
break; /// We've already created all the tables.
for (const auto & table_name : tables_to_create)
auto tables_to_create = tables_dependencies.getTablesSortedByDependency();
for (const auto & table_id : tables_to_create)
{
auto table_name = table_id.getQualifiedName();
createTable(table_name);
checkTable(table_name);
insertDataToTable(table_name);
}
}
}
void RestorerFromBackup::createTable(const QualifiedTableName & table_name)
@ -752,62 +796,6 @@ void RestorerFromBackup::insertDataToTable(const QualifiedTableName & table_name
}
}
/// Returns the list of tables without dependencies or those which dependencies have been created before.
std::vector<QualifiedTableName> RestorerFromBackup::findTablesWithoutDependencies() const
{
std::vector<QualifiedTableName> tables_without_dependencies;
bool all_tables_created = true;
for (const auto & [key, table_info] : table_infos)
{
if (table_info.storage)
continue;
/// Found a table which is not created yet.
all_tables_created = false;
/// Check if all dependencies have been created before.
bool all_dependencies_met = true;
for (const auto & dependency : table_info.dependencies)
{
auto it = table_infos.find(dependency);
if ((it != table_infos.end()) && !it->second.storage)
{
all_dependencies_met = false;
break;
}
}
if (all_dependencies_met)
tables_without_dependencies.push_back(key);
}
if (!tables_without_dependencies.empty())
return tables_without_dependencies;
if (all_tables_created)
return {};
/// Cyclic dependency? We'll try to create those tables anyway but probably it's going to fail.
std::vector<QualifiedTableName> tables_with_cyclic_dependencies;
for (const auto & [key, table_info] : table_infos)
{
if (!table_info.storage)
tables_with_cyclic_dependencies.push_back(key);
}
/// Only show a warning here, proper exception will be thrown later on creating those tables.
LOG_WARNING(
log,
"Some tables have cyclic dependency from each other: {}",
boost::algorithm::join(
tables_with_cyclic_dependencies
| boost::adaptors::transformed([](const QualifiedTableName & table_name) -> String { return table_name.getFullName(); }),
", "));
return tables_with_cyclic_dependencies;
}
void RestorerFromBackup::addDataRestoreTask(DataRestoreTask && new_task)
{
if (current_stage == Stage::INSERTING_DATA_TO_TABLES)

View File

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

View File

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

View File

@ -2,7 +2,7 @@
#include <IO/WriteHelpers.h>
#include <Common/VariableContext.h>
#include <Interpreters/TraceCollector.h>
#include <Common/TraceSender.h>
#include <Common/Exception.h>
#include <Common/LockMemoryExceptionInThread.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))
{
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);
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)))
{
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;
}
@ -305,7 +305,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT
if (peak_updated && allocation_traced)
{
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))
@ -361,7 +361,7 @@ void MemoryTracker::free(Int64 size)
if (unlikely(sample_probability > 0.0 && sample(thread_local_rng)))
{
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;

View File

@ -1,5 +1,6 @@
#include <Common/ProfileEvents.h>
#include <Common/CurrentThread.h>
#include <Common/TraceSender.h>
/// Available events. Add something here as you wish.
@ -523,15 +524,29 @@ const char * getDocumentation(Event event)
return strings[event];
}
Event end() { return END; }
void increment(Event event, Count 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)
{
init();

View File

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

View File

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

View File

@ -14,7 +14,7 @@ namespace
/// The performance test query ids can be surprisingly long like
/// `aggregating_merge_tree_simple_aggregate_function_string.query100.profile100`,
/// 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());
}
@ -23,7 +23,7 @@ namespace DB
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
+ 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(TraceType) /// trace type
+ 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
/// (since recursive collect() is possible)
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];
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(thread_id, out);
writePODBinary(size, out);
writePODBinary(extras.size, out);
writePODBinary(extras.event, out);
writePODBinary(extras.increment, out);
out.next();
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <Common/PipeFDs.h>
#include <Common/ProfileEvents.h>
#include <base/types.h>
class StackTrace;
@ -17,6 +18,7 @@ enum class TraceType : uint8_t
Memory,
MemorySample,
MemoryPeak,
ProfileEvent,
};
/// This is the second part of TraceCollector, that sends stacktrace to the pipe.
@ -24,10 +26,18 @@ enum class TraceType : uint8_t
class TraceSender
{
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.
/// 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, Int64 size);
static void send(TraceType trace_type, const StackTrace & stack_trace, Extras extras);
private:
friend class TraceCollector;

View File

@ -64,11 +64,11 @@ bool enoughSpaceInDirectory(const std::string & path, size_t data_size)
return data_size <= free_space;
}
std::unique_ptr<PocoTemporaryFile> createTemporaryFile(const std::string & folder_path)
std::unique_ptr<TemporaryFile> createTemporaryFile(const std::string & path)
{
ProfileEvents::increment(ProfileEvents::ExternalProcessingFilesTotal);
fs::create_directories(folder_path);
return std::make_unique<PocoTemporaryFile>(folder_path);
fs::create_directories(path);
return std::make_unique<TemporaryFile>(path);
}
#if !defined(OS_LINUX)

View File

@ -14,10 +14,10 @@ namespace fs = std::filesystem;
namespace DB
{
using PocoTemporaryFile = Poco::TemporaryFile;
using TemporaryFile = Poco::TemporaryFile;
bool enoughSpaceInDirectory(const std::string & path, size_t data_size);
std::unique_ptr<PocoTemporaryFile> createTemporaryFile(const std::string & folder_path);
std::unique_ptr<TemporaryFile> createTemporaryFile(const std::string & path);
// Determine what block device is responsible for specified path

View File

@ -280,6 +280,7 @@ Changelog::Changelog(
, force_sync(force_sync_)
, log(log_)
, compress_logs(compress_logs_)
, write_operations(std::numeric_limits<size_t>::max())
{
/// Load all files in changelog directory
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());
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)
{
std::lock_guard writer_lock(writer_mutex);
std::optional<ChangelogReadResult> last_log_read_result;
/// Last log has some free space to write
@ -336,7 +340,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin
removeAllLogs();
min_log_id = last_commited_log_index;
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;
}
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".
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();
}
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();
if (current_writer)
current_writer->flush(force_sync);
/// Start new one
ChangelogFileDescription new_description;
@ -540,27 +547,72 @@ ChangelogRecord Changelog::buildRecord(uint64_t index, const LogEntryPtr & log_e
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)
{
if (!current_writer)
if (!initialized)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Changelog must be initialized before appending records");
if (logs.empty())
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;
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)
{
{
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();
@ -586,6 +638,7 @@ void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry)
itr = existing_changelogs.erase(itr);
}
}
}
/// Remove redundant logs from memory
/// Everything >= index must be removed
@ -597,6 +650,7 @@ void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry)
void Changelog::compact(uint64_t up_to_log_index)
{
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);
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; });
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);
}
@ -747,8 +801,19 @@ void Changelog::applyEntriesFromBuffer(uint64_t index, nuraft::buffer & buffer)
void Changelog::flush()
{
if (current_writer)
current_writer->flush(force_sync);
if (flushAsync())
{
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()
@ -758,6 +823,12 @@ void Changelog::shutdown()
if (clean_log_thread.joinable())
clean_log_thread.join();
if (!write_operations.isFinished())
write_operations.finish();
if (write_thread.joinable())
write_thread.join();
}
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
#include <libnuraft/nuraft.hxx>
#include <libnuraft/raft_server.hxx>
#include <city.h>
#include <optional>
#include <base/defines.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/HashingWriteBuffer.h>
#include <IO/CompressionMethod.h>
@ -121,6 +123,8 @@ public:
/// Fsync latest log to disk and flush buffer
void flush();
bool flushAsync();
void shutdown();
uint64_t size() const
@ -128,6 +132,14 @@ public:
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
~Changelog();
@ -136,7 +148,7 @@ private:
static ChangelogRecord buildRecord(uint64_t index, const LogEntryPtr & log_entry);
/// 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
std::map<uint64_t, ChangelogFileDescription> existing_changelogs;
@ -162,7 +174,7 @@ private:
Poco::Logger * log;
bool compress_logs;
std::mutex writer_mutex;
/// Current writer for changelog file
std::unique_ptr<ChangelogWriter> current_writer;
/// Mapping log_id -> log_entry
@ -175,6 +187,33 @@ private:
/// 128 is enough, even if log is not removed, it's not a problem
ConcurrentBoundedQueue<std::string> log_files_to_delete_queue{128};
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*/)
{
std::lock_guard lock(changelog_lock);
changelog.flush();
changelog.flushAsync();
}
nuraft::ptr<nuraft::log_entry> KeeperLogStore::getLatestConfigChange() const
@ -132,4 +132,16 @@ bool KeeperLogStore::flushChangelogAndShutdown()
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
uint64_t size() const;
uint64_t last_durable_index() override;
/// Flush batch of appended entries
void end_of_append_batch(uint64_t start_index, uint64_t count) override;
/// Get entry with latest config in logstore
nuraft::ptr<nuraft::log_entry> getLatestConfigChange() const;
void setRaftServer(const nuraft::ptr<nuraft::raft_server> & raft_server);
private:
mutable std::mutex changelog_lock;
Poco::Logger * log;

View File

@ -266,6 +266,7 @@ void KeeperServer::forceRecovery()
void KeeperServer::launchRaftServer(const Poco::Util::AbstractConfiguration & config, bool enable_ipv6)
{
nuraft::raft_params params;
params.parallel_log_appending_ = true;
params.heart_beat_interval_
= getValueOrMaxInt32AndLogWarning(coordination_settings->heart_beat_interval_ms.totalMilliseconds(), "heart_beat_interval_ms", log);
params.election_timeout_lower_bound_ = getValueOrMaxInt32AndLogWarning(
@ -352,6 +353,8 @@ void KeeperServer::launchRaftServer(const Poco::Util::AbstractConfiguration & co
if (!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_);
nuraft::ptr<nuraft::raft_server> casted_raft_server = raft_instance;
@ -446,8 +449,8 @@ void KeeperServer::shutdownRaftServer()
void KeeperServer::shutdown()
{
state_manager->flushAndShutDownLogStore();
shutdownRaftServer();
state_manager->flushAndShutDownLogStore();
state_machine->shutdownStorage();
}

View File

@ -67,6 +67,7 @@ class CoordinationTest : public ::testing::TestWithParam<CompressionParam>
{
protected:
DB::KeeperContextPtr keeper_context = std::make_shared<DB::KeeperContext>();
Poco::Logger * log{&Poco::Logger::get("CoordinationTest")};
};
TEST_P(CoordinationTest, BuildTest)
@ -129,10 +130,13 @@ struct SimpliestRaftServer
params.snapshot_distance_ = 1; /// forcefully send snapshots
params.client_req_timeout_ = 3000;
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(
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)
{
@ -140,6 +144,10 @@ struct SimpliestRaftServer
_exit(1);
}
state_manager->getLogStore()->setRaftServer(raft_instance);
raft_instance->start_server(false);
std::cout << "init Raft instance " << server_id;
for (size_t ii = 0; ii < 20; ++ii)
{
@ -207,7 +215,7 @@ TEST_P(CoordinationTest, TestSummingRaft1)
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));
}
@ -240,6 +248,15 @@ TEST_P(CoordinationTest, ChangelogTestSimple)
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)
{
@ -250,6 +267,9 @@ TEST_P(CoordinationTest, ChangelogTestFile)
auto entry = getLogEntry("hello world", 77);
changelog.append(entry);
changelog.end_of_append_batch(0, 0);
waitDurableLogs(changelog);
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension));
for (const auto & p : fs::directory_iterator("./logs"))
EXPECT_EQ(p.path(), "./logs/changelog_1_5.bin" + params.extension);
@ -261,6 +281,8 @@ TEST_P(CoordinationTest, ChangelogTestFile)
changelog.append(entry);
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_6_10.bin" + params.extension));
}
@ -271,6 +293,7 @@ TEST_P(CoordinationTest, ChangelogReadWrite)
ChangelogDirTest test("./logs");
DB::KeeperLogStore changelog("./logs", 1000, true, params.enable_compression);
changelog.init(1, 0);
for (size_t i = 0; i < 10; ++i)
{
auto entry = getLogEntry("hello world", i * 10);
@ -280,6 +303,8 @@ TEST_P(CoordinationTest, ChangelogReadWrite)
EXPECT_EQ(changelog.size(), 10);
waitDurableLogs(changelog);
DB::KeeperLogStore changelog_reader("./logs", 1000, true, params.enable_compression);
changelog_reader.init(1, 0);
EXPECT_EQ(changelog_reader.size(), 10);
@ -315,6 +340,8 @@ TEST_P(CoordinationTest, ChangelogWriteAt)
changelog.write_at(7, entry);
changelog.end_of_append_batch(0, 0);
waitDurableLogs(changelog);
EXPECT_EQ(changelog.size(), 7);
EXPECT_EQ(changelog.last_entry()->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);
EXPECT_EQ(changelog.size(), 7);
waitDurableLogs(changelog);
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.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);
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_6_10.bin" + params.extension));
@ -371,6 +403,8 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead)
changelog_reader.append(entry);
changelog_reader.end_of_append_batch(0, 0);
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_6_10.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);
waitDurableLogs(changelog);
EXPECT_EQ(changelog.size(), 3);
changelog.compact(2);
@ -416,6 +452,8 @@ TEST_P(CoordinationTest, ChangelogTestCompaction)
changelog.append(e4);
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_6_10.bin" + params.extension));
@ -454,6 +492,8 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperations)
EXPECT_EQ(changelog.size(), 10);
waitDurableLogs(changelog);
auto entries = changelog.pack(1, 5);
DB::KeeperLogStore apply_changelog("./logs", 100, true, params.enable_compression);
@ -499,6 +539,8 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperationsEmpty)
EXPECT_EQ(changelog.size(), 10);
waitDurableLogs(changelog);
auto entries = changelog.pack(5, 5);
ChangelogDirTest test1("./logs1");
@ -543,6 +585,8 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtPreviousFile)
}
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_6_10.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.last_entry()->get_term(), 5555);
waitDurableLogs(changelog);
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.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);
waitDurableLogs(changelog);
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_11_15.bin" + params.extension));
@ -610,6 +658,8 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtFileBorder)
EXPECT_EQ(changelog.next_slot(), 12);
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_6_10.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");
DB::KeeperLogStore changelog("./logs", 5, true, params.enable_compression);
changelog.init(1, 0);
for (size_t i = 0; i < 33; ++i)
{
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);
waitDurableLogs(changelog);
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_11_15.bin" + params.extension));
@ -659,6 +710,8 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtAllFiles)
EXPECT_EQ(changelog.next_slot(), 2);
EXPECT_EQ(changelog.last_entry()->get_term(), 5555);
waitDurableLogs(changelog);
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.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);
EXPECT_EQ(changelog.size(), 35);
waitDurableLogs(changelog);
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_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_FALSE(fs::exists("./logs/changelog_36_40.bin" + params.extension));
DB::KeeperLogStore changelog_reader("./logs", 5, true, params.enable_compression);
changelog_reader.init(1, 0);
@ -701,6 +755,8 @@ TEST_P(CoordinationTest, ChangelogTestStartNewLogAfterRead)
changelog_reader.end_of_append_batch(0, 0);
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_6_10.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);
EXPECT_EQ(changelog.size(), 35);
waitDurableLogs(changelog);
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_11_15.bin" + params.extension));
@ -779,6 +837,8 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate)
EXPECT_EQ(changelog_reader.size(), 11);
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_6_10.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);
waitDurableLogs(changelog);
EXPECT_TRUE(fs::exists("./logs/changelog_1_20.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);
changelog_reader.append(entry);
changelog_reader.end_of_append_batch(0, 0);
waitDurableLogs(changelog_reader);
EXPECT_EQ(changelog_reader.size(), 1);
EXPECT_EQ(changelog_reader.last_entry()->get_term(), 7777);
@ -848,6 +912,7 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles)
}
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_21_40.bin" + params.extension));
@ -874,6 +939,8 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles2)
}
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_11_20.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.end_of_append_batch(0, 0);
waitDurableLogs(changelog);
state_machine->pre_commit(i, changelog.entry_at(i)->get_buf());
state_machine->commit(i, changelog.entry_at(i)->get_buf());
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*/)
{
snapshot_created = ret;
std::cerr << "Snapshot finished\n";
LOG_INFO(&Poco::Logger::get("CoordinationTest"), "Snapshot finished");
};
state_machine->create_snapshot(s, when_done);
@ -1511,6 +1580,8 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges)
changelog.append(entry);
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);
}
waitDurableLogs(changelog_1);
EXPECT_TRUE(fs::exists("./logs/changelog_1_100.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);
}
waitDurableLogs(changelog_2);
changelog_2.compact(105);
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);
}
waitDurableLogs(changelog_3);
changelog_3.compact(125);
std::this_thread::sleep_for(std::chrono::microseconds(1000));
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);
}
waitDurableLogs(changelog);
DB::KeeperLogStore changelog1("./logs", 100, true, test_params.enable_compression);
changelog1.init(0, 3);
@ -1683,43 +1761,47 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth)
auto params = GetParam();
ChangelogDirTest test("./logs");
{
std::cerr << "================First time=====================\n";
LOG_INFO(log, "================First time=====================");
DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression);
changelog.init(1, 0);
auto entry = getLogEntry("hello_world", 1000);
changelog.append(entry);
changelog.end_of_append_batch(0, 0);
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);
changelog.init(1, 0);
auto entry = getLogEntry("hello_world", 1000);
changelog.append(entry);
changelog.end_of_append_batch(0, 0);
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);
changelog.init(1, 0);
auto entry = getLogEntry("hello_world", 1000);
changelog.append(entry);
changelog.end_of_append_batch(0, 0);
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);
changelog.init(1, 0);
auto entry = getLogEntry("hello_world", 1000);
changelog.append(entry);
changelog.end_of_append_batch(0, 0);
EXPECT_EQ(changelog.next_slot(), 5);
waitDurableLogs(changelog);
}
}
@ -1730,7 +1812,7 @@ TEST_P(CoordinationTest, ChangelogInsertMultipleTimesSmooth)
ChangelogDirTest test("./logs");
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);
changelog.init(1, 0);
for (size_t j = 0; j < 7; ++j)
@ -1739,6 +1821,7 @@ TEST_P(CoordinationTest, ChangelogInsertMultipleTimesSmooth)
changelog.append(entry);
}
changelog.end_of_append_batch(0, 0);
waitDurableLogs(changelog);
}
DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression);
@ -1750,37 +1833,49 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard)
{
auto params = GetParam();
ChangelogDirTest test("./logs");
std::cerr << "================First time=====================\n";
{
LOG_INFO(log, "================First time=====================");
DB::KeeperLogStore changelog1("./logs", 100, true, params.enable_compression);
changelog1.init(1, 0);
auto entry = getLogEntry("hello_world", 1000);
changelog1.append(entry);
changelog1.end_of_append_batch(0, 0);
EXPECT_EQ(changelog1.next_slot(), 2);
waitDurableLogs(changelog1);
}
std::cerr << "================Second time=====================\n";
{
LOG_INFO(log, "================Second time=====================");
DB::KeeperLogStore changelog2("./logs", 100, true, params.enable_compression);
changelog2.init(1, 0);
entry = getLogEntry("hello_world", 1000);
auto entry = getLogEntry("hello_world", 1000);
changelog2.append(entry);
changelog2.end_of_append_batch(0, 0);
EXPECT_EQ(changelog2.next_slot(), 3);
waitDurableLogs(changelog2);
}
std::cerr << "================Third time=====================\n";
{
LOG_INFO(log, "================Third time=====================");
DB::KeeperLogStore changelog3("./logs", 100, true, params.enable_compression);
changelog3.init(1, 0);
entry = getLogEntry("hello_world", 1000);
auto entry = getLogEntry("hello_world", 1000);
changelog3.append(entry);
changelog3.end_of_append_batch(0, 0);
EXPECT_EQ(changelog3.next_slot(), 4);
waitDurableLogs(changelog3);
}
std::cerr << "================Fourth time=====================\n";
{
LOG_INFO(log, "================Fourth time=====================");
DB::KeeperLogStore changelog4("./logs", 100, true, params.enable_compression);
changelog4.init(1, 0);
entry = getLogEntry("hello_world", 1000);
auto entry = getLogEntry("hello_world", 1000);
changelog4.append(entry);
changelog4.end_of_append_batch(0, 0);
EXPECT_EQ(changelog4.next_slot(), 5);
waitDurableLogs(changelog4);
}
}
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, 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(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) \
\
@ -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_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(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(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)
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.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"},

View File

@ -303,17 +303,17 @@ public:
*/
/// There is two variants for binary serde. First variant work with Field.
virtual void serializeBinary(const Field & field, WriteBuffer & ostr) const = 0;
virtual void deserializeBinary(Field & field, ReadBuffer & istr) const = 0;
virtual void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const = 0;
virtual void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const = 0;
/// Other variants takes a column, to avoid creating temporary Field object.
/// Column must be non-constant.
/// 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.
/// 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.
*/

View File

@ -17,13 +17,13 @@
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 &>();
writeBinary(state.data, ostr);
}
void SerializationAggregateFunction::deserializeBinary(Field & field, ReadBuffer & istr) const
void SerializationAggregateFunction::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const
{
field = AggregateFunctionStateData();
AggregateFunctionStateData & s = field.get<AggregateFunctionStateData &>();
@ -31,12 +31,12 @@ void SerializationAggregateFunction::deserializeBinary(Field & field, ReadBuffer
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);
}
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);

View File

@ -22,11 +22,11 @@ public:
: function(function_), type_name(std::move(type_name_)), version(version_) {}
/// NOTE These two functions for serializing single values are incompatible with the functions below.
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) 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 deserializeBinary(IColumn & column, ReadBuffer & istr) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) 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 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;

View File

@ -27,18 +27,18 @@ static constexpr size_t MAX_ARRAY_SIZE = 1ULL << 30;
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 &>();
writeVarUInt(a.size(), ostr);
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;
readVarUInt(size, istr);
@ -46,11 +46,11 @@ void SerializationArray::deserializeBinary(Field & field, ReadBuffer & istr) con
Array & arr = field.get<Array &>();
arr.reserve(size);
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::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();
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::Offsets & offsets = column_array.getOffsets();
@ -81,7 +81,7 @@ void SerializationArray::deserializeBinary(IColumn & column, ReadBuffer & istr)
try
{
for (; i < size; ++i)
nested->deserializeBinary(nested_column, istr);
nested->deserializeBinary(nested_column, istr, settings);
}
catch (...)
{

View File

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

View File

@ -12,14 +12,14 @@ namespace DB
{
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>>();
writeBinary(x, ostr);
}
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);
writeBinary(x, ostr);
@ -39,7 +39,7 @@ void SerializationDecimalBase<T>::serializeBinaryBulk(const IColumn & column, Wr
}
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;
readBinary(x, istr);
@ -47,7 +47,7 @@ void SerializationDecimalBase<T>::deserializeBinary(Field & field, ReadBuffer &
}
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;
readBinary(x, istr);

View File

@ -20,12 +20,12 @@ public:
SerializationDecimalBase(UInt32 precision_, UInt32 scale_)
: precision(precision_), scale(scale_) {}
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
void serializeBinary(const IColumn & column, size_t row_num, 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 FormatSettings &) 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(IColumn & column, ReadBuffer & istr) const override;
void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) 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;
};

View File

@ -26,7 +26,7 @@ namespace ErrorCodes
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 &>();
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();
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);
}
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();
size_t old_size = data.size();

View File

@ -15,10 +15,10 @@ public:
explicit SerializationFixedString(size_t n_) : n(n_) {}
size_t getN() const { return n; }
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) 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 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);
}
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

View File

@ -49,10 +49,10 @@ public:
DeserializeBinaryBulkStatePtr & state,
SubstreamsCache * cache) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) 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 FormatSettings & settings) 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 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;

View File

@ -36,7 +36,7 @@ static IColumn & extractNestedColumn(IColumn & column)
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 &>();
writeVarUInt(map.size(), ostr);
@ -44,12 +44,12 @@ void SerializationMap::serializeBinary(const Field & field, WriteBuffer & ostr)
{
const auto & tuple = elem.safeGet<const Tuple>();
assert(tuple.size() == 2);
key->serializeBinary(tuple[0], ostr);
value->serializeBinary(tuple[1], ostr);
key->serializeBinary(tuple[0], ostr, settings);
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;
readVarUInt(size, istr);
@ -59,20 +59,20 @@ void SerializationMap::deserializeBinary(Field & field, ReadBuffer & istr) const
for (size_t i = 0; i < size; ++i)
{
Tuple tuple(2);
key->deserializeBinary(tuple[0], istr);
value->deserializeBinary(tuple[1], istr);
key->deserializeBinary(tuple[0], istr, settings);
value->deserializeBinary(tuple[1], istr, settings);
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:
SerializationMap(const SerializationPtr & key_type_, const SerializationPtr & value_type_, const SerializationPtr & nested_);
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) 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 FormatSettings & settings) 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 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;

View File

@ -19,10 +19,10 @@ private:
throw Exception("Serialization is not implemented for type Nothing", ErrorCodes::NOT_IMPLEMENTED);
}
public:
void serializeBinary(const Field &, WriteBuffer &) const override { throwNoSerialization(); }
void deserializeBinary(Field &, ReadBuffer &) const override { throwNoSerialization(); }
void serializeBinary(const IColumn &, size_t, WriteBuffer &) const override { throwNoSerialization(); }
void deserializeBinary(IColumn &, ReadBuffer &) const override { throwNoSerialization(); }
void serializeBinary(const Field &, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void deserializeBinary(Field &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
void serializeBinary(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) 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 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())
{
@ -159,17 +159,17 @@ void SerializationNullable::serializeBinary(const Field & field, WriteBuffer & o
else
{
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;
readBinary(is_null, istr);
if (!is_null)
{
nested->deserializeBinary(field, istr);
nested->deserializeBinary(field, istr, settings);
}
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);
bool is_null = col.isNullAt(row_num);
writeBinary(is_null, ostr);
if (!is_null)
nested->serializeBinary(col.getNestedColumn(), row_num, ostr);
nested->serializeBinary(col.getNestedColumn(), row_num, ostr, settings);
}
/// 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,
[&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,
SubstreamsCache * cache) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) 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 FormatSettings & settings) 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 deserializeTextEscaped(IColumn & column, ReadBuffer & istr, 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>
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
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>
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;
readBinary(x, istr);
@ -118,13 +118,13 @@ void SerializationNumber<T>::deserializeBinary(Field & field, ReadBuffer & istr)
}
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);
}
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;
readBinary(x, istr);

View File

@ -22,10 +22,10 @@ public:
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
/** Format is platform-dependent. */
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) 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 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>
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");
}
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");
}
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");
}
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");
}

View File

@ -57,10 +57,10 @@ public:
DeserializeBinaryBulkStatePtr & state,
SubstreamsCache * cache) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) 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 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.
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);
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");
}

View File

@ -61,11 +61,11 @@ public:
DeserializeBinaryBulkStatePtr & state,
SubstreamsCache * cache) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) 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 deserializeBinary(IColumn & column, ReadBuffer & istr) 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 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;

View File

@ -25,20 +25,37 @@ namespace DB
namespace ErrorCodes
{
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 &>();
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);
writeString(s, ostr);
}
void SerializationString::deserializeBinary(Field & field, ReadBuffer & istr) const
void SerializationString::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const
{
UInt64 size;
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();
String & s = field.get<String &>();
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);
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);
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::Chars & data = column_string.getChars();
@ -62,6 +87,13 @@ void SerializationString::deserializeBinary(IColumn & column, ReadBuffer & istr)
UInt64 size;
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 offset = old_chars_size + size + 1;

View File

@ -8,10 +8,10 @@ namespace DB
class SerializationString final : public ISerialization
{
public:
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) 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 FormatSettings & settings) 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 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);
}
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 &>();
for (size_t element_index = 0; element_index < elems.size(); ++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();
@ -47,15 +47,15 @@ void SerializationTuple::deserializeBinary(Field & field, ReadBuffer & istr) con
Tuple & tuple = field.get<Tuple &>();
tuple.reserve(size);
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)
{
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, [&]
{
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 deserializeBinary(Field & field, ReadBuffer & istr) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) 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 FormatSettings & settings) 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 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;

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>();
writeBinary(x, ostr);
}
void SerializationUUID::deserializeBinary(Field & field, ReadBuffer & istr) const
void SerializationUUID::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const
{
UUID x;
readBinary(x, istr);
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);
}
void SerializationUUID::deserializeBinary(IColumn & column, ReadBuffer & istr) const
void SerializationUUID::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{
UUID x;
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 deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) 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 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);
}
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

View File

@ -55,11 +55,11 @@ public:
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 serializeBinary(const Field & field, WriteBuffer & ostr) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) 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 deserializeBinary(IColumn & column, ReadBuffer & istr) 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 FormatSettings & settings) 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;

View File

@ -1,117 +1,85 @@
#include <Databases/DDLDependencyVisitor.h>
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Poco/String.h>
namespace DB
{
using TableLoadingDependenciesVisitor = DDLDependencyVisitor::Visitor;
TableNamesSet getDependenciesSetFromCreateQuery(ContextPtr global_context, const QualifiedTableName & table, const ASTPtr & ast)
namespace
{
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 DDLDependencyVisitor::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>())
/// CREATE TABLE or CREATE DICTIONARY or CREATE VIEW or CREATE TEMPORARY TABLE or CREATE DATABASE query.
void visitCreateQuery(const ASTCreateQuery & create, DDLDependencyVisitor::Data & data)
{
if (child.get() == create->select)
return false;
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);
}
return true;
}
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);
}
}
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)
/// 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;
auto config = getDictionaryConfigurationFromAST(data.create_query->as<ASTCreateQuery &>(), data.global_context);
auto info = getInfoIfClickHouseDictionarySource(config, data.global_context);
if (!info || !info->is_local)
const ASTIdentifier * identifier = dynamic_cast<const ASTIdentifier *>(expr.database_and_table_name.get());
if (!identifier)
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")
auto table_identifier = identifier->createTable();
if (!table_identifier)
return;
extractTableNameFromArgument(*storage.engine, data, 0);
}
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;
}
void DDLDependencyVisitor::extractTableNameFromArgument(const ASTFunction & function, Data & data, size_t arg_idx)
{
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 * arg = function.arguments->as<ASTExpressionList>()->children[arg_idx].get();
const auto * expr_list = function.arguments->as<ASTExpressionList>();
if (!expr_list)
return;
const auto * arg = expr_list->children[arg_idx].get();
if (const auto * literal = arg->as<ASTLiteral>())
{
if (literal->value.getType() != Field::Types::String)
@ -137,7 +105,7 @@ void DDLDependencyVisitor::extractTableNameFromArgument(const ASTFunction & func
}
else
{
assert(false);
/// Just return because we don't validate AST in this function.
return;
}
@ -147,52 +115,125 @@ void DDLDependencyVisitor::extractTableNameFromArgument(const ASTFunction & func
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);
}
/// 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());
DDLDependencyVisitor::Data data;
data.table_name = table_name;
data.default_database = global_context->getCurrentDatabase();
data.create_query = ast;
data.global_context = global_context;
DDLDependencyVisitor::Visitor visitor{data};
visitor.visit(ast);
data.dependencies.erase(data.table_name);
return data.dependencies;
}
void DDLDependencyVisitor::visit(const ASTPtr & ast, Data & data)
{
if (auto * create = ast->as<ASTCreateQuery>())
{
visitCreateQuery(*create, data);
}
else if (auto * dictionary = ast->as<ASTDictionary>())
{
visitDictionaryDef(*dictionary, data);
}
else if (auto * expr = ast->as<ASTTableExpression>())
{
visitTableExpression(*expr, data);
}
else if (const auto * function = ast->as<ASTFunction>())
{
if (function->kind == ASTFunction::Kind::TABLE_ENGINE)
visitTableEngine(*function, data);
else
visitFunction(*function, data);
}
}
bool DDLDependencyVisitor::needChildVisit(const ASTPtr &, const ASTPtr &)
{
return true;
}
}

View File

@ -1,72 +1,36 @@
#pragma once
#include <Core/QualifiedTableName.h>
#include <Parsers/IAST_fwd.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Core/QualifiedTableName.h>
namespace DB
{
class ASTFunction;
class ASTFunctionWithKeyValueArguments;
class ASTStorage;
using TableNamesSet = std::unordered_set<QualifiedTableName>;
TableNamesSet getDependenciesSetFromCreateQuery(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 names of table (or dictionary) dependencies
/// from column default expressions (joinGet, dictGet, etc)
/// or dictionary source (for dictionaries from local ClickHouse table).
/// 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.
/// 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:
struct Data
{
String default_database;
TableNamesSet dependencies;
ContextPtr global_context;
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);
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);
static bool needChildVisit(const ASTPtr & node, const ASTPtr & child);
};
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 <Databases/DatabaseMemory.h>
#include <Databases/DatabasesCommon.h>
#include <Databases/DDLDependencyVisitor.h>
#include <Databases/DDLLoadingDependencyVisitor.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTCreateQuery.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());
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

View File

@ -4,7 +4,7 @@
#include <Databases/DatabaseOnDisk.h>
#include <Databases/DatabaseOrdinary.h>
#include <Databases/DatabasesCommon.h>
#include <Databases/DDLDependencyVisitor.h>
#include <Databases/DDLLoadingDependencyVisitor.h>
#include <Databases/TablesLoader.h>
#include <IO/ReadBufferFromFile.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()};
TableNamesSet loading_dependencies = getDependenciesSetFromCreateQuery(getContext(), qualified_name, ast);
std::lock_guard lock{metadata.mutex};
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;
}
}
@ -321,8 +309,8 @@ void DatabaseOrdinary::alterTable(ContextPtr local_context, const StorageID & ta
out.close();
}
TableNamesSet new_dependencies = getDependenciesSetFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), ast);
DatabaseCatalog::instance().updateLoadingDependencies(table_id, std::move(new_dependencies));
auto new_dependencies = getLoadingDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), ast);
DatabaseCatalog::instance().updateDependencies(table_id, new_dependencies);
commitAlterTable(table_id, table_metadata_tmp_path, table_metadata_path, statement, local_context);
}

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/IDatabase.h>
#include <Databases/DDLDependencyVisitor.h>
#include <Databases/DDLLoadingDependencyVisitor.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExternalDictionariesLoader.h>
@ -14,45 +15,12 @@ namespace DB
namespace ErrorCodes
{
extern const int INFINITE_LOOP;
extern const int LOGICAL_ERROR;
}
static constexpr size_t PRINT_MESSAGE_EACH_N_OBJECTS = 256;
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)
{
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_)
, databases(std::move(databases_))
, strictness_mode(strictness_mode_)
, referential_dependencies("ReferentialDeps")
, loading_dependencies("LoadingDeps")
{
metadata.default_database = global_context->getCurrentDatabase();
log = &Poco::Logger::get("TablesLoader");
@ -101,20 +71,18 @@ void TablesLoader::loadTables()
stopwatch.restart();
logDependencyGraph();
/// Remove tables that do not exist
removeUnresolvableDependencies(/* remove_loaded */ false);
buildDependencyGraph();
/// 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.
removeUnresolvableDependencies(/* remove_loaded */ true);
/// Remove tables that do not exist
removeUnresolvableDependencies();
loadTablesInTopologicalOrder(pool);
}
void TablesLoader::startupTables()
{
/// 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
if (metadata.parsed_tables.contains(dependency_name))
if (metadata.parsed_tables.contains(table_id.getQualifiedName()))
return false;
/// Table exists and it's already loaded
if (DatabaseCatalog::instance().isTableExist(StorageID(dependency_name.database, dependency_name.table), global_context))
return remove_loaded;
/// It's XML dictionary.
if (dependency_name.database == metadata.default_database &&
global_context->getExternalDictionariesLoader().has(dependency_name.table))
if (DatabaseCatalog::instance().isTableExist(table_id, global_context))
{
LOG_WARNING(log, "Tables {} depend on XML dictionary {}, but XML dictionaries are loaded independently."
"Consider converting it to DDL dictionary.", fmt::join(info.dependent_database_objects, ", "), dependency_name);
return true;
/// Table exists and it's already loaded
}
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.
/// We will ignore it and try to load dependent tables without "dependency_name"
/// (but most likely dependent tables will fail to load).
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())
size_t num_dependencies, num_dependents;
loading_dependencies.getNumberOfAdjacents(table_id, num_dependencies, num_dependents);
if (num_dependencies || !num_dependents)
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();
while (table_it != metadata.dependencies_info.end())
{
auto & info = table_it->second;
if (need_exclude_dependency(table_it->first, info))
table_it = removeResolvedDependency(table_it, metadata.independent_database_objects);
else
++table_it;
}
loading_dependencies.removeTablesIf(need_exclude_dependency);
if (loading_dependencies.getNumberOfTables() != metadata.parsed_tables.size())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Number of tables to be loaded is not as expected. It's a bug");
/// Cannot load tables with cyclic dependencies.
loading_dependencies.checkNoCyclicDependencies();
}
void TablesLoader::loadTablesInTopologicalOrder(ThreadPool & pool)
{
/// 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);
load_context->setSetting("cast_ipv4_ipv6_default_on_conversion_error", 1);
/// Load independent tables in parallel.
/// Then remove loaded tables from dependency graph, find tables/dictionaries that do not have unresolved dependencies anymore,
/// move them to the list of independent tables and load.
/// Repeat until we have some tables to load.
/// If we do not, then either all objects are loaded or there is cyclic dependency.
/// Complexity: O(V + E)
size_t level = 0;
do
{
assert(metadata.parsed_tables.size() == tables_processed + metadata.independent_database_objects.size() + getNumberOfTablesWithDependencies());
logDependencyGraph();
/// Load tables in parallel.
auto tables_to_load = loading_dependencies.getTablesSortedByDependencyForParallel();
startLoadingIndependentTables(pool, level, load_context);
TableNames new_independent_database_objects;
for (const auto & table_name : metadata.independent_database_objects)
for (size_t level = 0; level != tables_to_load.size(); ++level)
{
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);
}
startLoadingTables(pool, load_context, tables_to_load[level], level);
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();
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];
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/Types.h>
#include <Databases/LoadingStrictnessLevel.h>
#include <Databases/TablesDependencyGraph.h>
#include <Interpreters/Context_fwd.h>
#include <Parsers/IAST_fwd.h>
#include <Common/Stopwatch.h>
@ -34,21 +35,6 @@ struct 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
{
@ -59,17 +45,6 @@ struct ParsedTablesMetadata
/// For logging
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
@ -92,25 +67,18 @@ private:
Strings databases_to_load;
ParsedTablesMetadata metadata;
TablesDependencyGraph referential_dependencies;
TablesDependencyGraph loading_dependencies;
Poco::Logger * log;
std::atomic<size_t> tables_processed{0};
AtomicStopwatch stopwatch;
ThreadPool pool;
void removeUnresolvableDependencies(bool remove_loaded);
void buildDependencyGraph();
void removeUnresolvableDependencies();
void loadTablesInTopologicalOrder(ThreadPool & pool);
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;
void startLoadingTables(ThreadPool & pool, ContextMutablePtr load_context, const std::vector<StorageID> & tables_to_load, size_t level);
};
}

View File

@ -118,7 +118,10 @@ void CachedOnDiskReadBufferFromFile::initialize(size_t offset, size_t size)
}
else
{
CreateFileSegmentSettings create_settings(is_persistent ? FileSegmentKind::Persistent : FileSegmentKind::Regular);
CreateFileSegmentSettings create_settings{
.is_persistent = is_persistent
};
file_segments_holder.emplace(cache->getOrSet(cache_key, offset, size, create_settings));
}
@ -948,7 +951,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
}
else
{
LOG_TRACE(log, "No space left in cache to reserve {} bytes, will continue without cache download", size);
LOG_TRACE(log, "No space left in cache, will continue without cache download");
file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
}

View File

@ -51,42 +51,16 @@ FileSegmentRangeWriter::FileSegmentRangeWriter(
{
}
bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind)
{
size_t written_size = tryWrite(data, size, offset, segment_kind, true);
return written_size == size;
}
size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind, bool strict)
{
size_t total_written_size = 0;
while (size > 0)
{
size_t written_size = tryWriteImpl(data, size, offset, segment_kind, strict);
chassert(written_size <= size);
if (written_size == 0)
break;
if (data)
data += written_size;
size -= written_size;
offset += written_size;
total_written_size += written_size;
}
return total_written_size;
}
size_t FileSegmentRangeWriter::tryWriteImpl(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind, bool strict)
bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset, bool is_persistent)
{
if (finalized)
return 0;
return false;
auto & file_segments = file_segments_holder.file_segments;
if (current_file_segment_it == file_segments.end())
{
current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, segment_kind);
current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, is_persistent);
}
else
{
@ -104,7 +78,7 @@ size_t FileSegmentRangeWriter::tryWriteImpl(const char * data, size_t size, size
if (file_segment->range().size() == file_segment->getDownloadedSize())
{
completeFileSegment(*file_segment);
current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, segment_kind);
current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, is_persistent);
}
}
@ -119,26 +93,20 @@ size_t FileSegmentRangeWriter::tryWriteImpl(const char * data, size_t size, size
file_segment->completePartAndResetDownloader();
});
size_t reserved_size = file_segment->tryReserve(size, strict);
if (reserved_size == 0 || (strict && reserved_size != size))
{
if (strict)
bool reserved = file_segment->reserve(size);
if (!reserved)
{
file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
appendFilesystemCacheLog(*file_segment);
}
LOG_DEBUG(
&Poco::Logger::get("FileSegmentRangeWriter"),
"Unsuccessful space reservation attempt (size: {}, file segment info: {}",
size, file_segment->getInfoForLog());
return 0;
return false;
}
/// Shrink to reserved size, because we can't write more than reserved
size = reserved_size;
try
{
file_segment->write(data, size, offset);
@ -152,17 +120,7 @@ size_t FileSegmentRangeWriter::tryWriteImpl(const char * data, size_t size, size
file_segment->completePartAndResetDownloader();
current_file_segment_write_offset += size;
return size;
}
bool FileSegmentRangeWriter::reserve(size_t size, size_t offset)
{
return write(nullptr, size, offset, FileSegmentKind::Temporary);
}
size_t FileSegmentRangeWriter::tryReserve(size_t size, size_t offset)
{
return tryWrite(nullptr, size, offset, FileSegmentKind::Temporary);
return true;
}
void FileSegmentRangeWriter::finalize()
@ -171,7 +129,6 @@ void FileSegmentRangeWriter::finalize()
return;
auto & file_segments = file_segments_holder.file_segments;
if (file_segments.empty() || current_file_segment_it == file_segments.end())
return;
@ -192,7 +149,7 @@ FileSegmentRangeWriter::~FileSegmentRangeWriter()
}
}
FileSegments::iterator FileSegmentRangeWriter::allocateFileSegment(size_t offset, FileSegmentKind segment_kind)
FileSegments::iterator FileSegmentRangeWriter::allocateFileSegment(size_t offset, bool is_persistent)
{
/**
* Allocate a new file segment starting `offset`.
@ -201,7 +158,10 @@ FileSegments::iterator FileSegmentRangeWriter::allocateFileSegment(size_t offset
std::lock_guard cache_lock(cache->mutex);
CreateFileSegmentSettings create_settings(segment_kind);
CreateFileSegmentSettings create_settings
{
.is_persistent = is_persistent,
};
/// We set max_file_segment_size to be downloaded,
/// if we have less size to write, file segment will be resized in complete() method.
@ -236,15 +196,12 @@ void FileSegmentRangeWriter::appendFilesystemCacheLog(const FileSegment & file_s
}
}
void FileSegmentRangeWriter::completeFileSegment(FileSegment & file_segment, std::optional<FileSegment::State> state)
void FileSegmentRangeWriter::completeFileSegment(FileSegment & file_segment)
{
/// File segment can be detached if space reservation failed.
if (file_segment.isDetached())
return;
if (state.has_value())
file_segment.setDownloadState(*state);
file_segment.completeWithoutState();
appendFilesystemCacheLog(file_segment);
}
@ -313,8 +270,7 @@ void CachedOnDiskWriteBufferFromFile::cacheData(char * data, size_t size)
try
{
auto segment_kind = is_persistent_cache_file ? FileSegmentKind::Persistent : FileSegmentKind::Regular;
if (!cache_writer->write(data, size, current_download_offset, segment_kind))
if (!cache_writer->write(data, size, current_download_offset, is_persistent_cache_file))
{
LOG_INFO(log, "Write-through cache is stopped as cache limit is reached and nothing can be evicted");
return;

View File

@ -4,7 +4,6 @@
#include <IO/WriteSettings.h>
#include <Interpreters/Cache/FileCache.h>
#include <Interpreters/FilesystemCacheLog.h>
#include <Common/filesystemHelpers.h>
namespace Poco
{
@ -29,44 +28,22 @@ public:
FileCache * cache_, const FileSegment::Key & key_,
std::shared_ptr<FilesystemCacheLog> cache_log_, const String & query_id_, const String & source_path_);
/* Write a range of file segments.
* Allocate file segment of `max_file_segment_size` and write to it until it is full and then allocate next file segment.
* If it's impossible to allocate new file segment and reserve space to write all data, then returns false.
*
* Note: the data that was written to file segments before the error occurred is not rolled back.
/**
* Write a range of file segments. Allocate file segment of `max_file_segment_size` and write to
* it until it is full and then allocate next file segment.
*/
bool write(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind);
/* Tries to write data to current file segment.
* Size of written data may be less than requested_size, because it may not be enough space.
*
* Returns size of written data.
*/
size_t tryWrite(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind = FileSegmentKind::Regular, bool strict = false);
/// Same as `write/tryWrite`, but doesn't write anything, just reserves some space in cache
bool reserve(size_t size, size_t offset);
size_t tryReserve(size_t size, size_t offset);
bool write(const char * data, size_t size, size_t offset, bool is_persistent);
void finalize();
size_t currentOffset() const { return current_file_segment_write_offset; }
~FileSegmentRangeWriter();
private:
FileSegments::iterator allocateFileSegment(size_t offset, FileSegmentKind segment_kind);
FileSegments::iterator allocateFileSegment(size_t offset, bool is_persistent);
void appendFilesystemCacheLog(const FileSegment & file_segment);
void completeFileSegment(FileSegment & file_segment, std::optional<FileSegment::State> state = {});
/* Writes data to current file segment as much as possible and returns size of written data, do not allocate new file segments
* In `strict` mode it will write all data or nothing, otherwise it will write as much as possible
* If returned non zero value, then we can try to write again to next file segment.
* If no space is available, returns zero.
*/
size_t tryWriteImpl(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind, bool strict);
void completeFileSegment(FileSegment & file_segment);
FileCache * cache;
FileSegment::Key key;

View File

@ -1,73 +0,0 @@
#include <Disks/IO/FileCachePlaceholder.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_ENOUGH_SPACE;
}
void ISpacePlaceholder::reserveCapacity(size_t requested_capacity)
{
chassert(used_space <= capacity);
size_t remaining_space = capacity - used_space;
LOG_TEST(&Poco::Logger::get("ISpacePlaceholder"), "Reserving {} bytes (used_space: {}, capacity: {})", requested_capacity, used_space, capacity);
if (requested_capacity <= remaining_space)
return;
size_t capacity_to_reserve = requested_capacity - remaining_space;
reserveImpl(capacity_to_reserve);
capacity += capacity_to_reserve;
}
void ISpacePlaceholder::setUsed(size_t size)
{
LOG_TEST(&Poco::Logger::get("ISpacePlaceholder"), "Using {} bytes ({} already used, {} capacity)", size, used_space, capacity);
if (used_space + size > capacity)
{
LOG_WARNING(&Poco::Logger::get("ISpacePlaceholder"), "Used space is greater than capacity. It may lead to not enough space error");
reserveCapacity(size);
}
used_space = used_space + size;
}
FileCachePlaceholder::FileCachePlaceholder(FileCache * cache, const String & name)
: key_name(name)
, file_cache(cache)
{
}
void FileCachePlaceholder::reserveImpl(size_t requested_size)
{
/// We create new cache_writer and will try to reserve requested_size in it
String key = fmt::format("{}_{}", key_name, cache_writers.size());
auto cache_writer = std::make_unique<FileSegmentRangeWriter>(file_cache,
file_cache->hash(key),
/* cache_log_ */ nullptr,
/* query_id_ */ "",
/* source_path_ */ key);
size_t current_offset = cache_writer->currentOffset();
size_t reserved_size = cache_writer->tryReserve(requested_size, current_offset);
if (reserved_size != requested_size)
{
throw Exception(ErrorCodes::NOT_ENOUGH_SPACE,
"Cannot reserve space in file cache "
"({} bytes required, got {} reserved "
"{} / {} bytes used, "
"{} / {} elements used)"
, requested_size, reserved_size
, file_cache->getUsedCacheSize(), file_cache->getTotalMaxSize()
, file_cache->getFileSegmentsNum(), file_cache->getTotalMaxElements());
}
/// Add to cache_writers only if we successfully reserved space, otherwise free reserved_size back
cache_writers.push_back(std::move(cache_writer));
}
}

View File

@ -1,61 +0,0 @@
#pragma once
#include <Interpreters/Cache/FileCache.h>
#include <Disks/IO/CachedOnDiskWriteBufferFromFile.h>
#include <Poco/Logger.h>
#include <Poco/ConsoleChannel.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
/* ISpacePlaceholder is a base class for all classes that need to reserve space in some storage.
* You should resrve space with call reserveCapacity() before writing to it.
* After writing you should call setUsed() to let ISpacePlaceholder know how much space was used.
* It can be different because in some cases you don't know exact size of data you will write (because of compression, for example).
* It's better to reserve more space in advance not to overuse space.
*/
class ISpacePlaceholder
{
public:
/// Reserve space in storage
void reserveCapacity(size_t requested_capacity);
/// Indicate that some space is used
/// It uses reserved space if it is possible, otherwise it reserves more space
void setUsed(size_t size);
virtual ~ISpacePlaceholder() = default;
private:
virtual void reserveImpl(size_t size) = 0;
size_t capacity = 0;
size_t used_space = 0;
};
/* FileCachePlaceholder is a class that reserves space in FileCache.
* Data is written externally, and FileCachePlaceholder is only used to hold space in FileCache.
*/
class FileCachePlaceholder : public ISpacePlaceholder
{
public:
FileCachePlaceholder(FileCache * cache, const String & name);
void reserveImpl(size_t requested_size) override;
private:
std::string key_name;
FileCache * file_cache;
/// On each reserveImpl() call we create new FileSegmentRangeWriter that would be hold space
/// It's required to easily release already reserved space on unsuccessful attempt
std::vector<std::unique_ptr<FileSegmentRangeWriter>> cache_writers;
};
}

View File

@ -113,8 +113,6 @@ public:
WriteSettings getAdjustedSettingsFromMetadataFile(const WriteSettings & settings, const std::string & path) const override;
FileCachePtr getCache() const { return cache; }
private:
FileCache::Key getCacheKey(const std::string & path) const;

View File

@ -519,14 +519,6 @@ void DiskObjectStorage::wrapWithCache(FileCachePtr cache, const FileCacheSetting
object_storage = std::make_shared<CachedObjectStorage>(object_storage, cache, cache_settings, layer_name);
}
FileCachePtr DiskObjectStorage::getCache() const
{
const auto * cached_object_storage = typeid_cast<CachedObjectStorage *>(object_storage.get());
if (!cached_object_storage)
return nullptr;
return cached_object_storage->getCache();
}
NameSet DiskObjectStorage::getCacheLayersNames() const
{
NameSet cache_layers;

View File

@ -186,7 +186,6 @@ public:
/// There can be any number of cache layers:
/// DiskObjectStorage(CachedObjectStorage(...CacheObjectStorage(S3ObjectStorage)...))
void wrapWithCache(FileCachePtr cache, const FileCacheSettings & cache_settings, const String & layer_name);
FileCachePtr getCache() const;
/// Get structure of object storage this disk works with. Examples:
/// DiskObjectStorage(S3ObjectStorage)

View File

@ -1,20 +0,0 @@
#include <Disks/TemporaryFileInPath.h>
#include <Common/filesystemHelpers.h>
namespace DB
{
TemporaryFileInPath::TemporaryFileInPath(const String & folder_path)
: tmp_file(createTemporaryFile(folder_path))
{
chassert(tmp_file);
}
String TemporaryFileInPath::getPath() const
{
return tmp_file->path();
}
TemporaryFileInPath::~TemporaryFileInPath() = default;
}

View File

@ -1,21 +0,0 @@
#pragma once
#include <Disks/TemporaryFileOnDisk.h>
#include <Poco/TemporaryFile.h>
namespace DB
{
/// Wrapper around Poco::TemporaryFile to implement ITemporaryFile.
class TemporaryFileInPath : public ITemporaryFile
{
public:
explicit TemporaryFileInPath(const String & folder_path);
String getPath() const override;
~TemporaryFileInPath() override;
private:
std::unique_ptr<Poco::TemporaryFile> tmp_file;
};
}

View File

@ -2,7 +2,6 @@
#include <Poco/TemporaryFile.h>
#include <Common/CurrentMetrics.h>
#include <Common/logger_useful.h>
#include <Disks/TemporaryFileInPath.h>
#include <filesystem>
@ -16,6 +15,7 @@ namespace CurrentMetrics
extern const Metric TotalTemporaryFiles;
}
namespace DB
{

View File

@ -9,30 +9,21 @@ namespace DB
{
using DiskPtr = std::shared_ptr<IDisk>;
class ITemporaryFile
{
public:
virtual String getPath() const = 0;
virtual ~ITemporaryFile() = default;
};
using TemporaryFileHolder = std::unique_ptr<ITemporaryFile>;
/// This class helps with the handling of temporary files or directories.
/// A unique name for the temporary file or directory is automatically chosen based on a specified prefix.
/// Create a directory in the constructor.
/// The destructor always removes the temporary file or directory with all contained files.
class TemporaryFileOnDisk : public ITemporaryFile
class TemporaryFileOnDisk
{
public:
explicit TemporaryFileOnDisk(const DiskPtr & disk_);
explicit TemporaryFileOnDisk(const DiskPtr & disk_, CurrentMetrics::Value metric_scope);
explicit TemporaryFileOnDisk(const DiskPtr & disk_, const String & prefix);
~TemporaryFileOnDisk() override;
~TemporaryFileOnDisk();
DiskPtr getDisk() const { return disk; }
String getPath() const override;
String getPath() const;
private:
DiskPtr disk;

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.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.max_binary_string_size = settings.format_binary_max_string_size;
/// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context
if (format_settings.schema.is_server)

View File

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

View File

@ -64,10 +64,8 @@ static void writeData(const ISerialization & serialization, const ColumnPtr & co
}
size_t NativeWriter::write(const Block & block)
void NativeWriter::write(const Block & block)
{
size_t written_before = ostr.count();
/// Additional information about the block.
if (client_revision > 0)
block.info.write(ostr);
@ -163,10 +161,6 @@ size_t NativeWriter::write(const Block & block)
if (index)
index->blocks.emplace_back(std::move(index_block));
size_t written_after = ostr.count();
size_t written_size = written_after - written_before;
return written_size;
}
}

View File

@ -27,9 +27,7 @@ public:
IndexForNativeFormat * index_ = nullptr, size_t initial_size_of_file_ = 0);
Block getHeader() const { return header; }
/// Returns the number of bytes written.
size_t write(const Block & block);
void write(const Block & block);
void flush();
static String getContentType() { return "application/octet-stream"; }

View File

@ -27,14 +27,14 @@ struct TranslateImpl
const std::string & map_to)
{
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);
for (size_t i = 0; i < map_from.size(); ++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];
}
@ -125,7 +125,7 @@ struct TranslateUTF8Impl
auto map_to_size = UTF8::countCodePoints(reinterpret_cast<const UInt8 *>(map_to.data()), 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);

View File

@ -13,7 +13,7 @@ namespace ErrorCodes
}
WriteBufferFromTemporaryFile::WriteBufferFromTemporaryFile(std::unique_ptr<PocoTemporaryFile> && tmp_file_)
WriteBufferFromTemporaryFile::WriteBufferFromTemporaryFile(std::unique_ptr<TemporaryFile> && tmp_file_)
: WriteBufferFromFile(tmp_file_->path(), DBMS_DEFAULT_BUFFER_SIZE, O_RDWR | O_TRUNC | O_CREAT, 0600), tmp_file(std::move(tmp_file_))
{}
@ -40,11 +40,11 @@ public:
return std::make_shared<ReadBufferFromTemporaryWriteBuffer>(fd, file_name, std::move(origin->tmp_file));
}
ReadBufferFromTemporaryWriteBuffer(int fd_, const std::string & file_name_, std::unique_ptr<PocoTemporaryFile> && tmp_file_)
ReadBufferFromTemporaryWriteBuffer(int fd_, const std::string & file_name_, std::unique_ptr<TemporaryFile> && tmp_file_)
: ReadBufferFromFile(fd_, file_name_), tmp_file(std::move(tmp_file_))
{}
std::unique_ptr<PocoTemporaryFile> tmp_file;
std::unique_ptr<TemporaryFile> tmp_file;
};

View File

@ -20,11 +20,11 @@ public:
~WriteBufferFromTemporaryFile() override;
private:
explicit WriteBufferFromTemporaryFile(std::unique_ptr<PocoTemporaryFile> && tmp_file);
explicit WriteBufferFromTemporaryFile(std::unique_ptr<TemporaryFile> && tmp_file);
std::shared_ptr<ReadBuffer> getReadBufferImpl() override;
std::unique_ptr<PocoTemporaryFile> tmp_file;
std::unique_ptr<TemporaryFile> tmp_file;
friend class ReadBufferFromTemporaryWriteBuffer;
};

View File

@ -2,7 +2,6 @@
#include <Common/randomSeed.h>
#include <Common/SipHash.h>
#include <Common/logger_useful.h>
#include <Interpreters/Cache/FileCacheSettings.h>
#include <Interpreters/Cache/LRUFileCachePriority.h>
#include <IO/ReadHelpers.h>
@ -13,7 +12,6 @@
#include <pcg-random/pcg_random.hpp>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
@ -48,27 +46,13 @@ FileCache::Key FileCache::hash(const String & path)
return Key(sipHash128(path.data(), path.size()));
}
String FileCache::getPathInLocalCache(const Key & key, size_t offset, FileSegmentKind segment_kind) const
String FileCache::getPathInLocalCache(const Key & key, size_t offset, bool is_persistent) const
{
String file_suffix;
switch (segment_kind)
{
case FileSegmentKind::Persistent:
file_suffix = "_persistent";
break;
case FileSegmentKind::Temporary:
file_suffix = "_temporary";
break;
case FileSegmentKind::Regular:
file_suffix = "";
break;
}
auto key_str = key.toString();
return fs::path(cache_base_path)
/ key_str.substr(0, 3)
/ key_str
/ (std::to_string(offset) + file_suffix);
/ (std::to_string(offset) + (is_persistent ? "_persistent" : ""));
}
String FileCache::getPathInLocalCache(const Key & key) const
@ -556,6 +540,9 @@ FileSegmentPtr FileCache::createFileSegmentForDownload(
assertCacheCorrectness(key, cache_lock);
#endif
if (size > max_file_segment_size)
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Requested size exceeds max file segment size");
auto * cell = getCell(key, offset, cache_lock);
if (cell)
throw Exception(
@ -1012,17 +999,9 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard<std::mutex> & cache_lock
fs::directory_iterator key_it{key_prefix_it->path()};
for (; key_it != fs::directory_iterator(); ++key_it)
{
if (key_it->is_regular_file())
{
if (key_prefix_it->path().filename() == "tmp" && startsWith(key_it->path().filename(), "tmp"))
{
LOG_DEBUG(log, "Found temporary file '{}', will remove it", key_it->path().string());
fs::remove(key_it->path());
}
else
if (!key_it->is_directory())
{
LOG_DEBUG(log, "Unexpected file: {}. Expected a directory", key_it->path().string());
}
continue;
}
@ -1030,26 +1009,17 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard<std::mutex> & cache_lock
fs::directory_iterator offset_it{key_it->path()};
for (; offset_it != fs::directory_iterator(); ++offset_it)
{
if (offset_it->is_directory())
{
LOG_DEBUG(log, "Unexpected directory: {}. Expected a file", offset_it->path().string());
continue;
}
auto offset_with_suffix = offset_it->path().filename().string();
auto delim_pos = offset_with_suffix.find('_');
bool parsed;
FileSegmentKind segment_kind = FileSegmentKind::Regular;
bool is_persistent = false;
if (delim_pos == std::string::npos)
parsed = tryParse<UInt64>(offset, offset_with_suffix);
else
{
parsed = tryParse<UInt64>(offset, offset_with_suffix.substr(0, delim_pos));
if (offset_with_suffix.substr(delim_pos+1) == "persistent")
segment_kind = FileSegmentKind::Persistent;
if (offset_with_suffix.substr(delim_pos+1) == "temporary")
segment_kind = FileSegmentKind::Temporary;
is_persistent = offset_with_suffix.substr(delim_pos+1) == "persistent";
}
if (!parsed)
@ -1069,7 +1039,7 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard<std::mutex> & cache_lock
{
auto * cell = addCell(
key, offset, size, FileSegment::State::DOWNLOADED,
CreateFileSegmentSettings(segment_kind), cache_lock);
CreateFileSegmentSettings{ .is_persistent = is_persistent }, cache_lock);
if (cell)
queue_entries.emplace_back(cell->queue_iterator, cell->file_segment);
@ -1181,7 +1151,7 @@ std::vector<String> FileCache::tryGetCachePaths(const Key & key)
for (const auto & [offset, cell] : cells_by_offset)
{
if (cell.file_segment->state() == FileSegment::State::DOWNLOADED)
cache_paths.push_back(getPathInLocalCache(key, offset, cell.file_segment->getKind()));
cache_paths.push_back(getPathInLocalCache(key, offset, cell.file_segment->isPersistent()));
}
return cache_paths;
@ -1203,16 +1173,6 @@ size_t FileCache::getAvailableCacheSizeUnlocked(std::lock_guard<std::mutex> & ca
return max_size - getUsedCacheSizeUnlocked(cache_lock);
}
size_t FileCache::getTotalMaxSize() const
{
return max_size;
}
size_t FileCache::getTotalMaxElements() const
{
return max_element_size;
}
size_t FileCache::getFileSegmentsNum() const
{
std::lock_guard cache_lock(mutex);

View File

@ -80,7 +80,7 @@ public:
static Key hash(const String & path);
String getPathInLocalCache(const Key & key, size_t offset, FileSegmentKind segment_kind) const;
String getPathInLocalCache(const Key & key, size_t offset, bool is_persistent) const;
String getPathInLocalCache(const Key & key) const;
@ -89,10 +89,8 @@ public:
size_t capacity() const { return max_size; }
size_t getUsedCacheSize() const;
size_t getTotalMaxSize() const;
size_t getFileSegmentsNum() const;
size_t getTotalMaxElements() const;
static bool isReadOnly();
@ -223,8 +221,6 @@ private:
FileSegmentCell * getCell(const Key & key, size_t offset, std::lock_guard<std::mutex> & cache_lock);
/// Returns non-owened pointer to the cell stored in the `files` map.
/// Doesn't reserve any space.
FileSegmentCell * addCell(
const Key & key,
size_t offset,

View File

@ -31,21 +31,14 @@ const FileCacheSettings & FileCacheFactory::getSettings(const std::string & cach
}
FileCachePtr FileCacheFactory::tryGet(const std::string & cache_base_path)
FileCachePtr FileCacheFactory::get(const std::string & cache_base_path)
{
std::lock_guard lock(mutex);
auto it = caches_by_path.find(cache_base_path);
if (it == caches_by_path.end())
return nullptr;
return it->second->cache;
}
FileCachePtr FileCacheFactory::get(const std::string & cache_base_path)
{
auto file_cache_ptr = tryGet(cache_base_path);
if (!file_cache_ptr)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No cache found by path: {}", cache_base_path);
return file_cache_ptr;
return it->second->cache;
}
FileCachePtr FileCacheFactory::getOrCreate(

View File

@ -33,7 +33,6 @@ public:
FileCachePtr getOrCreate(const std::string & cache_base_path, const FileCacheSettings & file_cache_settings, const std::string & name);
FileCachePtr tryGet(const std::string & cache_base_path);
FileCachePtr get(const std::string & cache_base_path);
CacheByBasePath getAll();

View File

@ -23,19 +23,6 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
String toString(FileSegmentKind type)
{
switch (type)
{
case FileSegmentKind::Regular:
return "Regular";
case FileSegmentKind::Persistent:
return "Persistent";
case FileSegmentKind::Temporary:
return "Temporary";
}
}
FileSegment::FileSegment(
size_t offset_,
size_t size_,
@ -52,7 +39,7 @@ FileSegment::FileSegment(
#else
, log(&Poco::Logger::get("FileSegment"))
#endif
, segment_kind(settings.type)
, is_persistent(settings.is_persistent)
{
/// On creation, file segment state can be EMPTY, DOWNLOADED, DOWNLOADING.
switch (download_state)
@ -86,8 +73,7 @@ FileSegment::FileSegment(
String FileSegment::getPathInLocalCache() const
{
chassert(cache);
return cache->getPathInLocalCache(key(), offset(), segment_kind);
return cache->getPathInLocalCache(key(), offset(), isPersistent());
}
FileSegment::State FileSegment::state() const
@ -323,7 +309,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset)
if (current_downloaded_size == range().size())
throw Exception(ErrorCodes::LOGICAL_ERROR, "File segment is already fully downloaded");
if (!cache_writer && from != nullptr)
if (!cache_writer)
{
if (current_downloaded_size > 0)
throw Exception(
@ -338,13 +324,10 @@ void FileSegment::write(const char * from, size_t size, size_t offset)
try
{
/// if `from` is nullptr, then we just allocate and hold space by current segment and it was (or would) be written outside
if (cache_writer && from != nullptr)
cache_writer->write(from, size);
std::unique_lock download_lock(download_mutex);
if (cache_writer && from != nullptr)
cache_writer->next();
downloaded_size += size;
@ -396,13 +379,6 @@ FileSegment::State FileSegment::wait()
}
bool FileSegment::reserve(size_t size_to_reserve)
{
size_t reserved = tryReserve(size_to_reserve, true);
assert(reserved == 0 || reserved == size_to_reserve);
return reserved == size_to_reserve;
}
size_t FileSegment::tryReserve(size_t size_to_reserve, bool strict)
{
if (!size_to_reserve)
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Zero space reservation is not allowed");
@ -418,16 +394,10 @@ size_t FileSegment::tryReserve(size_t size_to_reserve, bool strict)
expected_downloaded_size = getDownloadedSizeUnlocked(segment_lock);
if (expected_downloaded_size + size_to_reserve > range().size())
{
if (strict)
{
throw Exception(
ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR,
"Attempt to reserve space too much space ({}) for file segment with range: {} (downloaded size: {})",
size_to_reserve, range().toString(), downloaded_size);
}
size_to_reserve = range().size() - expected_downloaded_size;
}
chassert(reserved_size >= expected_downloaded_size);
}
@ -445,16 +415,17 @@ size_t FileSegment::tryReserve(size_t size_to_reserve, bool strict)
{
std::lock_guard cache_lock(cache->mutex);
size_t need_to_reserve = size_to_reserve - already_reserved_size;
reserved = cache->tryReserve(key(), offset(), need_to_reserve, cache_lock);
if (!reserved)
return 0;
size_to_reserve = size_to_reserve - already_reserved_size;
reserved = cache->tryReserve(key(), offset(), size_to_reserve, cache_lock);
if (reserved)
{
std::lock_guard segment_lock(mutex);
reserved_size += need_to_reserve;
reserved_size += size_to_reserve;
}
return size_to_reserve;
}
return reserved;
}
void FileSegment::setDownloadedUnlocked([[maybe_unused]] std::unique_lock<std::mutex> & segment_lock)
@ -574,15 +545,6 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard<std::mutex> & cach
resetDownloaderUnlocked(segment_lock);
}
if (segment_kind == FileSegmentKind::Temporary && is_last_holder)
{
LOG_TEST(log, "Removing temporary file segment: {}", getInfoForLogUnlocked(segment_lock));
detach(cache_lock, segment_lock);
setDownloadState(State::SKIP_CACHE);
cache->remove(key(), offset(), cache_lock, segment_lock);
return;
}
switch (download_state)
{
case State::SKIP_CACHE:
@ -664,7 +626,7 @@ String FileSegment::getInfoForLogUnlocked(std::unique_lock<std::mutex> & segment
info << "first non-downloaded offset: " << getFirstNonDownloadedOffsetUnlocked(segment_lock) << ", ";
info << "caller id: " << getCallerId() << ", ";
info << "detached: " << is_detached << ", ";
info << "kind: " << toString(segment_kind);
info << "persistent: " << is_persistent;
return info.str();
}
@ -759,7 +721,7 @@ FileSegmentPtr FileSegment::getSnapshot(const FileSegmentPtr & file_segment, std
snapshot->ref_count = file_segment.use_count();
snapshot->downloaded_size = file_segment->getDownloadedSizeUnlocked(segment_lock);
snapshot->download_state = file_segment->download_state;
snapshot->segment_kind = file_segment->getKind();
snapshot->is_persistent = file_segment->isPersistent();
return snapshot;
}
@ -821,8 +783,6 @@ FileSegmentsHolder::~FileSegmentsHolder()
if (!cache)
cache = file_segment->cache;
assert(cache == file_segment->cache); /// all segments should belong to the same cache
try
{
bool is_detached = false;

View File

@ -30,38 +30,9 @@ using FileSegmentPtr = std::shared_ptr<FileSegment>;
using FileSegments = std::list<FileSegmentPtr>;
/*
* FileSegmentKind is used to specify the eviction policy for file segments.
*/
enum class FileSegmentKind
{
/* `Regular` file segment is still in cache after usage, and can be evicted
* (unless there're some holders).
*/
Regular,
/* `Persistent` file segment can't be evicted from cache,
* it should be removed manually.
*/
Persistent,
/* `Temporary` file segment is removed right after relesing.
* Also corresponding files are removed during cache loading (if any).
*/
Temporary,
};
String toString(FileSegmentKind type);
struct CreateFileSegmentSettings
{
FileSegmentKind type = FileSegmentKind::Regular;
CreateFileSegmentSettings() = default;
explicit CreateFileSegmentSettings(FileSegmentKind type_)
: type(type_)
{}
bool is_persistent = false;
};
class FileSegment : private boost::noncopyable, public std::enable_shared_from_this<FileSegment>
@ -156,8 +127,7 @@ public:
size_t offset() const { return range().left; }
FileSegmentKind getKind() const { return segment_kind; }
bool isPersistent() const { return segment_kind == FileSegmentKind::Persistent; }
bool isPersistent() const { return is_persistent; }
using UniqueId = std::pair<FileCacheKey, size_t>;
UniqueId getUniqueId() const { return std::pair(key(), offset()); }
@ -213,19 +183,19 @@ public:
void assertCorrectness() const;
/**
* ========== Methods for _only_ file segment's `writer` ======================
*/
void synchronousWrite(const char * from, size_t size, size_t offset);
/**
* ========== Methods for _only_ file segment's `downloader` ==================
*/
/// Try to reserve exactly `size` bytes.
/// Returns true if reservation was successful, false otherwise.
bool reserve(size_t size_to_reserve);
/// Try to reserve at max `size` bytes.
/// Returns actual size reserved.
/// In strict mode throws an error on attempt to reserve space too much space
size_t tryReserve(size_t size_to_reserve, bool strict = false);
/// Write data into reserved space.
void write(const char * from, size_t size, size_t offset);
@ -277,9 +247,9 @@ private:
void assertIsDownloaderUnlocked(const std::string & operation, std::unique_lock<std::mutex> & segment_lock) const;
void assertCorrectnessUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
/// completeWithoutStateUnlocked() is called from destructor of FileSegmentsHolder.
/// Function might check if the caller of the method
/// is the last alive holder of the segment. Therefore, completion and destruction
/// complete() without any completion state is called from destructor of
/// FileSegmentsHolder. complete() might check if the caller of the method
/// is the last alive holder of the segment. Therefore, complete() and destruction
/// of the file segment pointer must be done under the same cache mutex.
void completeWithoutStateUnlocked(std::lock_guard<std::mutex> & cache_lock);
void completeBasedOnCurrentState(std::lock_guard<std::mutex> & cache_lock, std::unique_lock<std::mutex> & segment_lock);
@ -325,12 +295,12 @@ private:
/// In general case, all file segments are owned by cache.
bool is_detached = false;
bool is_downloaded = false;
bool is_downloaded{false};
std::atomic<size_t> hits_count = 0; /// cache hits.
std::atomic<size_t> ref_count = 0; /// Used for getting snapshot state
FileSegmentKind segment_kind;
bool is_persistent;
CurrentMetrics::Increment metric_increment{CurrentMetrics::CacheFileSegments};
};
@ -343,8 +313,6 @@ struct FileSegmentsHolder : private boost::noncopyable
FileSegmentsHolder(FileSegmentsHolder && other) noexcept : file_segments(std::move(other.file_segments)) {}
void reset() { file_segments.clear(); }
~FileSegmentsHolder();
String toString();

View File

@ -32,7 +32,6 @@
#include <Storages/StorageS3Settings.h>
#include <Disks/DiskLocal.h>
#include <Disks/DiskDecorator.h>
#include <Disks/ObjectStorages/DiskObjectStorage.h>
#include <Disks/ObjectStorages/IObjectStorage.h>
#include <Disks/IO/ThreadPoolRemoteFSReader.h>
#include <Disks/IO/ThreadPoolReader.h>
@ -103,7 +102,6 @@
#include <Interpreters/Lemmatizers.h>
#include <Interpreters/ClusterDiscovery.h>
#include <Interpreters/TransactionLog.h>
#include <Interpreters/Cache/FileCacheFactory.h>
#include <filesystem>
#include <re2/re2.h>
@ -748,65 +746,28 @@ void Context::setPath(const String & path)
shared->user_scripts_path = shared->path + "user_scripts/";
}
static void setupTmpPath(Poco::Logger * log, const std::string & path)
try
VolumePtr Context::setTemporaryStorage(const String & path, const String & policy_name, size_t max_size)
{
LOG_DEBUG(log, "Setting up {} to store temporary data in it", path);
std::lock_guard lock(shared->storage_policies_mutex);
VolumePtr volume;
fs::create_directories(path);
/// Clearing old temporary files.
fs::directory_iterator dir_end;
for (fs::directory_iterator it(path); it != dir_end; ++it)
if (policy_name.empty())
{
if (it->is_regular_file() && startsWith(it->path().filename(), "tmp"))
{
LOG_DEBUG(log, "Removing old temporary file {}", it->path().string());
fs::remove(it->path());
}
else
LOG_DEBUG(log, "Found unknown file in temporary path {}", it->path().string());
}
}
catch (...)
{
DB::tryLogCurrentException(log, fmt::format(
"Caught exception while setup temporary path: {}. "
"It is ok to skip this exception as cleaning old temporary files is not necessary", path));
}
static VolumePtr createLocalSingleDiskVolume(const std::string & path)
{
auto disk = std::make_shared<DiskLocal>("_tmp_default", path, 0);
VolumePtr volume = std::make_shared<SingleDiskVolume>("_tmp_default", disk, 0);
return volume;
}
void Context::setTemporaryStoragePath(const String & path, size_t max_size)
{
shared->tmp_path = path;
if (!shared->tmp_path.ends_with('/'))
shared->tmp_path += '/';
VolumePtr volume = createLocalSingleDiskVolume(shared->tmp_path);
for (const auto & disk : volume->getDisks())
{
setupTmpPath(shared->log, disk->getPath());
auto disk = std::make_shared<DiskLocal>("_tmp_default", shared->tmp_path, 0);
volume = std::make_shared<SingleDiskVolume>("_tmp_default", disk, 0);
}
shared->temp_data_on_disk = std::make_shared<TemporaryDataOnDiskScope>(volume, nullptr, max_size);
}
void Context::setTemporaryStoragePolicy(const String & policy_name, size_t max_size)
{
std::lock_guard lock(shared->storage_policies_mutex);
else
{
StoragePolicyPtr tmp_policy = getStoragePolicySelector(lock)->get(policy_name);
if (tmp_policy->getVolumes().size() != 1)
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG,
"Policy '{}' is used temporary files, such policy should have exactly one volume", policy_name);
VolumePtr volume = tmp_policy->getVolume(0);
volume = tmp_policy->getVolume(0);
}
if (volume->getDisks().empty())
throw Exception("No disks volume for temporary files", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
@ -828,33 +789,10 @@ void Context::setTemporaryStoragePolicy(const String & policy_name, size_t max_s
"Disk '{}' ({}) is not local and can't be used for temporary files",
disk_ptr->getName(), typeid(*disk_raw_ptr).name());
}
setupTmpPath(shared->log, disk->getPath());
}
shared->temp_data_on_disk = std::make_shared<TemporaryDataOnDiskScope>(volume, nullptr, max_size);
}
void Context::setTemporaryStorageInCache(const String & cache_disk_name, size_t max_size)
{
auto disk_ptr = getDisk(cache_disk_name);
if (!disk_ptr)
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Disk '{}' is not found", cache_disk_name);
const auto * disk_object_storage_ptr = dynamic_cast<const DiskObjectStorage *>(disk_ptr.get());
if (!disk_object_storage_ptr)
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Disk '{}' does not use cache", cache_disk_name);
auto file_cache = disk_object_storage_ptr->getCache();
if (!file_cache)
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Cache '{}' is not found", file_cache->getBasePath());
LOG_DEBUG(shared->log, "Using file cache ({}) for temporary files", file_cache->getBasePath());
shared->tmp_path = file_cache->getBasePath();
VolumePtr volume = createLocalSingleDiskVolume(shared->tmp_path);
shared->temp_data_on_disk = std::make_shared<TemporaryDataOnDiskScope>(volume, file_cache.get(), max_size);
shared->temp_data_on_disk = std::make_shared<TemporaryDataOnDiskScope>(volume, max_size);
return volume;
}
void Context::setFlagsPath(const String & path)

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