Merge branch 'master' into fix-incorrect-exception-message

This commit is contained in:
Alexander Gololobov 2022-12-05 14:58:18 +01:00 committed by GitHub
commit 13617530e2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
68 changed files with 2357 additions and 582 deletions

1
.gitignore vendored
View File

@ -17,6 +17,7 @@
# logs
*.log
*.debuglog
*.stderr
*.stdout

2
contrib/NuRaft vendored

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

View File

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

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

@ -37,7 +37,7 @@
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <TableFunctions/registerTableFunctions.h>
#include <Storages/registerStorages.h>
#include <Storages/NamedCollections.h>
#include <Storages/NamedCollectionUtils.h>
#include <Dictionaries/registerDictionaries.h>
#include <Disks/registerDisks.h>
#include <Formats/registerFormats.h>
@ -120,7 +120,7 @@ void LocalServer::initialize(Poco::Util::Application & self)
config().getUInt("max_io_thread_pool_free_size", 0),
config().getUInt("io_thread_pool_queue_size", 10000));
NamedCollectionFactory::instance().initialize(config());
NamedCollectionUtils::loadFromConfig(config());
}
@ -212,6 +212,8 @@ void LocalServer::tryInitPath()
global_context->setUserFilesPath(""); // user's files are everywhere
NamedCollectionUtils::loadFromSQL(global_context);
/// top_level_domains_lists
const std::string & top_level_domains_path = config().getString("top_level_domains_path", path + "top_level_domains/");
if (!top_level_domains_path.empty())

View File

@ -60,7 +60,7 @@
#include <Storages/System/attachInformationSchemaTables.h>
#include <Storages/Cache/ExternalDataSourceCache.h>
#include <Storages/Cache/registerRemoteFileMetadatas.h>
#include <Storages/NamedCollections.h>
#include <Storages/NamedCollectionUtils.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <Functions/UserDefined/IUserDefinedSQLObjectsLoader.h>
#include <Functions/registerFunctions.h>
@ -782,7 +782,7 @@ try
config().getUInt("max_io_thread_pool_free_size", 0),
config().getUInt("io_thread_pool_queue_size", 10000));
NamedCollectionFactory::instance().initialize(config());
NamedCollectionUtils::loadFromConfig(config());
/// Initialize global local cache for remote filesystem.
if (config().has("local_cache_for_remote_fs"))
@ -1168,6 +1168,8 @@ try
SensitiveDataMasker::setInstance(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
}
NamedCollectionUtils::loadFromSQL(global_context);
auto main_config_reloader = std::make_unique<ConfigReloader>(
config_path,
include_from_path,
@ -1336,7 +1338,8 @@ try
#if USE_SSL
CertificateReloader::instance().tryLoad(*config);
#endif
NamedCollectionFactory::instance().reload(*config);
NamedCollectionUtils::reloadFromConfig(*config);
ProfileEvents::increment(ProfileEvents::MainConfigLoads);
/// Must be the last.

View File

@ -104,7 +104,7 @@ public:
/// The same as allColumnFlags().
static AccessFlags allFlagsGrantableOnColumnLevel();
static constexpr size_t SIZE = 128;
static constexpr size_t SIZE = 256;
private:
using Flags = std::bitset<SIZE>;
Flags flags;

View File

@ -69,6 +69,7 @@ enum class AccessType
M(ALTER_FREEZE_PARTITION, "FREEZE PARTITION, UNFREEZE", TABLE, ALTER_TABLE) \
\
M(ALTER_DATABASE_SETTINGS, "ALTER DATABASE SETTING, ALTER MODIFY DATABASE SETTING, MODIFY DATABASE SETTING", DATABASE, ALTER_DATABASE) /* allows to execute ALTER MODIFY SETTING */\
M(ALTER_NAMED_COLLECTION, "", GROUP, ALTER) /* allows to execute ALTER NAMED COLLECTION */\
\
M(ALTER_TABLE, "", GROUP, ALTER) \
M(ALTER_DATABASE, "", GROUP, ALTER) \
@ -88,6 +89,7 @@ enum class AccessType
M(CREATE_TEMPORARY_TABLE, "", GLOBAL, CREATE) /* allows to create and manipulate temporary tables;
implicitly enabled by the grant CREATE_TABLE on any table */ \
M(CREATE_FUNCTION, "", GLOBAL, CREATE) /* allows to execute CREATE FUNCTION */ \
M(CREATE_NAMED_COLLECTION, "", GLOBAL, CREATE) /* allows to execute CREATE NAMED COLLECTION */ \
M(CREATE, "", GROUP, ALL) /* allows to execute {CREATE|ATTACH} */ \
\
M(DROP_DATABASE, "", DATABASE, DROP) /* allows to execute {DROP|DETACH} DATABASE */\
@ -96,6 +98,7 @@ enum class AccessType
implicitly enabled by the grant DROP_TABLE */\
M(DROP_DICTIONARY, "", DICTIONARY, DROP) /* allows to execute {DROP|DETACH} DICTIONARY */\
M(DROP_FUNCTION, "", GLOBAL, DROP) /* allows to execute DROP FUNCTION */\
M(DROP_NAMED_COLLECTION, "", GLOBAL, DROP) /* allows to execute DROP NAMED COLLECTION */\
M(DROP, "", GROUP, ALL) /* allows to execute {DROP|DETACH} */\
\
M(TRUNCATE, "TRUNCATE TABLE", TABLE, ALL) \

View File

@ -637,8 +637,9 @@
M(666, CANNOT_USE_CACHE) \
M(667, NOT_INITIALIZED) \
M(668, INVALID_STATE) \
M(669, UNKNOWN_NAMED_COLLECTION) \
M(669, NAMED_COLLECTION_DOESNT_EXIST) \
M(670, NAMED_COLLECTION_ALREADY_EXISTS) \
M(671, NAMED_COLLECTION_IS_IMMUTABLE) \
\
M(999, KEEPER_EXCEPTION) \
M(1000, POCO_EXCEPTION) \

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

@ -117,7 +117,7 @@ public:
WriteBuffer * working_buf = compressed_buffer ? compressed_buffer->getNestedBuffer() : file_buf.get();
/// Flush working buffer to file system
/// Flush working buffer to file system
working_buf->next();
/// Fsync file system if needed
@ -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,50 +547,96 @@ 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)
{
/// This write_at require to overwrite everything in this file and also in previous file(s)
const bool go_to_previous_file = index < current_writer->getStartIndex();
if (go_to_previous_file)
{
auto index_changelog = existing_changelogs.lower_bound(index);
std::lock_guard lock(writer_mutex);
/// This write_at require to overwrite everything in this file and also in previous file(s)
const bool go_to_previous_file = index < current_writer->getStartIndex();
ChangelogFileDescription description;
if (index_changelog->first == index) /// exactly this file starts from index
description = index_changelog->second;
else
description = std::prev(index_changelog)->second;
/// Initialize writer from this log file
current_writer = std::make_unique<ChangelogWriter>(description.path, WriteMode::Append, index_changelog->first);
/// Remove all subsequent files if overwritten something in previous one
auto to_remove_itr = existing_changelogs.upper_bound(index);
for (auto itr = to_remove_itr; itr != existing_changelogs.end();)
if (go_to_previous_file)
{
std::filesystem::remove(itr->second.path);
itr = existing_changelogs.erase(itr);
auto index_changelog = existing_changelogs.lower_bound(index);
ChangelogFileDescription description;
if (index_changelog->first == index) /// exactly this file starts from index
description = index_changelog->second;
else
description = std::prev(index_changelog)->second;
/// Initialize writer from this log file
current_writer = std::make_unique<ChangelogWriter>(description.path, WriteMode::Append, index_changelog->first);
/// Remove all subsequent files if overwritten something in previous one
auto to_remove_itr = existing_changelogs.upper_bound(index);
for (auto itr = to_remove_itr; itr != existing_changelogs.end();)
{
std::filesystem::remove(itr->second.path);
itr = existing_changelogs.erase(itr);
}
}
}
@ -597,6 +650,7 @@ void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry)
void Changelog::compact(uint64_t up_to_log_index)
{
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";
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);
{
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";
DB::KeeperLogStore changelog2("./logs", 100, true, params.enable_compression);
changelog2.init(1, 0);
entry = getLogEntry("hello_world", 1000);
changelog2.append(entry);
changelog2.end_of_append_batch(0, 0);
EXPECT_EQ(changelog2.next_slot(), 3);
{
LOG_INFO(log, "================Second time=====================");
DB::KeeperLogStore changelog2("./logs", 100, true, params.enable_compression);
changelog2.init(1, 0);
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";
DB::KeeperLogStore changelog3("./logs", 100, true, params.enable_compression);
changelog3.init(1, 0);
entry = getLogEntry("hello_world", 1000);
changelog3.append(entry);
changelog3.end_of_append_batch(0, 0);
EXPECT_EQ(changelog3.next_slot(), 4);
{
LOG_INFO(log, "================Third time=====================");
DB::KeeperLogStore changelog3("./logs", 100, true, params.enable_compression);
changelog3.init(1, 0);
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";
DB::KeeperLogStore changelog4("./logs", 100, true, params.enable_compression);
changelog4.init(1, 0);
entry = getLogEntry("hello_world", 1000);
changelog4.append(entry);
changelog4.end_of_append_batch(0, 0);
EXPECT_EQ(changelog4.next_slot(), 5);
{
LOG_INFO(log, "================Fourth time=====================");
DB::KeeperLogStore changelog4("./logs", 100, true, params.enable_compression);
changelog4.init(1, 0);
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) \
\

View File

@ -40,11 +40,14 @@ TablesDependencyGraph::TablesDependencyGraph(TablesDependencyGraph && src) noexc
TablesDependencyGraph & TablesDependencyGraph::operator=(const TablesDependencyGraph & src)
{
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;
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;
}
@ -358,7 +361,7 @@ size_t TablesDependencyGraph::removeIsolatedTables()
auto it = nodes.begin();
while (it != nodes.end())
{
auto current = (it++)->get();
auto * current = (it++)->get();
if (current->dependencies.empty() && current->dependents.empty())
{
removeNode(current);

View File

@ -0,0 +1,28 @@
#include <Interpreters/InterpreterAlterNamedCollectionQuery.h>
#include <Parsers/ASTAlterNamedCollectionQuery.h>
#include <Access/ContextAccess.h>
#include <Interpreters/Context.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Storages/NamedCollectionUtils.h>
namespace DB
{
BlockIO InterpreterAlterNamedCollectionQuery::execute()
{
auto current_context = getContext();
current_context->checkAccess(AccessType::ALTER_NAMED_COLLECTION);
const auto & query = query_ptr->as<const ASTAlterNamedCollectionQuery &>();
if (!query.cluster.empty())
{
DDLQueryOnClusterParams params;
return executeDDLQueryOnCluster(query_ptr, current_context, params);
}
NamedCollectionUtils::updateFromSQL(query, current_context);
return {};
}
}

View File

@ -0,0 +1,22 @@
#pragma once
#include <Interpreters/IInterpreter.h>
namespace DB
{
class Context;
class InterpreterAlterNamedCollectionQuery : public IInterpreter, WithMutableContext
{
public:
InterpreterAlterNamedCollectionQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_)
: WithMutableContext(context_), query_ptr(query_ptr_) {}
BlockIO execute() override;
private:
ASTPtr query_ptr;
};
}

View File

@ -0,0 +1,30 @@
#include <Interpreters/InterpreterCreateNamedCollectionQuery.h>
#include <Parsers/ASTCreateNamedCollectionQuery.h>
#include <Access/ContextAccess.h>
#include <Interpreters/Context.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Storages/NamedCollectionUtils.h>
namespace DB
{
BlockIO InterpreterCreateNamedCollectionQuery::execute()
{
auto current_context = getContext();
current_context->checkAccess(AccessType::CREATE_NAMED_COLLECTION);
const auto & query = query_ptr->as<const ASTCreateNamedCollectionQuery &>();
if (!query.cluster.empty())
{
DDLQueryOnClusterParams params;
return executeDDLQueryOnCluster(query_ptr, current_context, params);
}
NamedCollectionUtils::createFromSQL(query, current_context);
return {};
}
}

View File

@ -0,0 +1,23 @@
#pragma once
#include <Interpreters/IInterpreter.h>
namespace DB
{
class InterpreterCreateNamedCollectionQuery : public IInterpreter, WithMutableContext
{
public:
InterpreterCreateNamedCollectionQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_)
: WithMutableContext(context_), query_ptr(query_ptr_)
{
}
BlockIO execute() override;
private:
ASTPtr query_ptr;
};
}

View File

@ -0,0 +1,32 @@
#include <Interpreters/InterpreterDropNamedCollectionQuery.h>
#include <Parsers/ASTDropNamedCollectionQuery.h>
#include <Access/ContextAccess.h>
#include <Interpreters/Context.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Storages/NamedCollectionUtils.h>
namespace DB
{
BlockIO InterpreterDropNamedCollectionQuery::execute()
{
auto current_context = getContext();
current_context->checkAccess(AccessType::DROP_NAMED_COLLECTION);
const auto & query = query_ptr->as<const ASTDropNamedCollectionQuery &>();
if (!query.cluster.empty())
{
DDLQueryOnClusterParams params;
return executeDDLQueryOnCluster(query_ptr, current_context, params);
}
if (query.if_exists)
NamedCollectionUtils::removeIfExistsFromSQL(query.collection_name, current_context);
else
NamedCollectionUtils::removeFromSQL(query.collection_name, current_context);
return {};
}
}

View File

@ -0,0 +1,22 @@
#pragma once
#include <Interpreters/IInterpreter.h>
namespace DB
{
class Context;
class InterpreterDropNamedCollectionQuery : public IInterpreter, WithMutableContext
{
public:
InterpreterDropNamedCollectionQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_)
: WithMutableContext(context_), query_ptr(query_ptr_) {}
BlockIO execute() override;
private:
ASTPtr query_ptr;
};
}

View File

@ -21,6 +21,9 @@
#include <Parsers/ASTShowTablesQuery.h>
#include <Parsers/ASTUseQuery.h>
#include <Parsers/ASTWatchQuery.h>
#include <Parsers/ASTCreateNamedCollectionQuery.h>
#include <Parsers/ASTDropNamedCollectionQuery.h>
#include <Parsers/ASTAlterNamedCollectionQuery.h>
#include <Parsers/MySQL/ASTCreateQuery.h>
#include <Parsers/ASTTransactionControl.h>
#include <Parsers/TablePropertiesQueriesASTs.h>
@ -47,6 +50,9 @@
#include <Interpreters/InterpreterCreateFunctionQuery.h>
#include <Interpreters/InterpreterCreateIndexQuery.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/InterpreterCreateNamedCollectionQuery.h>
#include <Interpreters/InterpreterDropNamedCollectionQuery.h>
#include <Interpreters/InterpreterAlterNamedCollectionQuery.h>
#include <Interpreters/InterpreterDeleteQuery.h>
#include <Interpreters/InterpreterDescribeQuery.h>
#include <Interpreters/InterpreterDescribeCacheQuery.h>
@ -230,6 +236,10 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, ContextMut
{
return std::make_unique<InterpreterAlterQuery>(query, context);
}
else if (query->as<ASTAlterNamedCollectionQuery>())
{
return std::make_unique<InterpreterAlterNamedCollectionQuery>(query, context);
}
else if (query->as<ASTCheckQuery>())
{
return std::make_unique<InterpreterCheckQuery>(query, context);
@ -270,6 +280,10 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, ContextMut
{
return std::make_unique<InterpreterDropAccessEntityQuery>(query, context);
}
else if (query->as<ASTDropNamedCollectionQuery>())
{
return std::make_unique<InterpreterDropNamedCollectionQuery>(query, context);
}
else if (query->as<ASTGrantQuery>())
{
return std::make_unique<InterpreterGrantQuery>(query, context);
@ -314,6 +328,10 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, ContextMut
{
return std::make_unique<InterpreterCreateIndexQuery>(query, context);
}
else if (query->as<ASTCreateNamedCollectionQuery>())
{
return std::make_unique<InterpreterCreateNamedCollectionQuery>(query, context);
}
else if (query->as<ASTDropIndexQuery>())
{
return std::make_unique<InterpreterDropIndexQuery>(query, context);

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -0,0 +1,54 @@
#include <Common/quoteString.h>
#include <Common/FieldVisitorToString.h>
#include <IO/Operators.h>
#include <Parsers/ASTAlterNamedCollectionQuery.h>
#include <Parsers/formatSettingName.h>
namespace DB
{
ASTPtr ASTAlterNamedCollectionQuery::clone() const
{
return std::make_shared<ASTAlterNamedCollectionQuery>(*this);
}
void ASTAlterNamedCollectionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "Alter NAMED COLLECTION ";
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);
if (!changes.empty())
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " SET " << (settings.hilite ? hilite_none : "");
bool first = true;
for (const auto & change : changes)
{
if (!first)
settings.ostr << ", ";
else
first = false;
formatSettingName(change.name, settings.ostr);
if (settings.show_secrets)
settings.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value);
else
settings.ostr << " = '[HIDDEN]'";
}
}
if (!delete_keys.empty())
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " DELETE " << (settings.hilite ? hilite_none : "");
bool first = true;
for (const auto & key : delete_keys)
{
if (!first)
settings.ostr << ", ";
else
first = false;
formatSettingName(key, settings.ostr);
}
}
}
}

View File

@ -0,0 +1,28 @@
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTQueryWithOnCluster.h>
#include <Common/SettingsChanges.h>
namespace DB
{
class ASTAlterNamedCollectionQuery : public IAST, public ASTQueryWithOnCluster
{
public:
std::string collection_name;
SettingsChanges changes;
std::vector<std::string> delete_keys;
bool if_exists = false;
String getID(char) const override { return "AlterNamedCollectionQuery"; }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster<ASTAlterNamedCollectionQuery>(clone()); }
};
}

View File

@ -0,0 +1,43 @@
#include <Common/quoteString.h>
#include <IO/Operators.h>
#include <Parsers/ASTCreateNamedCollectionQuery.h>
#include <Parsers/formatSettingName.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/FieldVisitorToString.h>
namespace DB
{
ASTPtr ASTCreateNamedCollectionQuery::clone() const
{
return std::make_shared<ASTCreateNamedCollectionQuery>(*this);
}
void ASTCreateNamedCollectionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE NAMED COLLECTION ";
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "");
bool first = true;
for (const auto & change : changes)
{
if (!first)
settings.ostr << ", ";
else
first = false;
formatSettingName(change.name, settings.ostr);
if (settings.show_secrets)
settings.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value);
else
settings.ostr << " = '[HIDDEN]'";
}
}
}

View File

@ -0,0 +1,28 @@
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTQueryWithOnCluster.h>
#include <Common/SettingsChanges.h>
namespace DB
{
class ASTCreateNamedCollectionQuery : public IAST, public ASTQueryWithOnCluster
{
public:
std::string collection_name;
SettingsChanges changes;
String getID(char) const override { return "CreateNamedCollectionQuery"; }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster<ASTCreateNamedCollectionQuery>(clone()); }
std::string getCollectionName() const;
};
}

View File

@ -0,0 +1,20 @@
#include <Parsers/ASTDropNamedCollectionQuery.h>
#include <Common/quoteString.h>
#include <IO/Operators.h>
namespace DB
{
ASTPtr ASTDropNamedCollectionQuery::clone() const
{
return std::make_shared<ASTDropNamedCollectionQuery>(*this);
}
void ASTDropNamedCollectionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP NAMED COLLECTION ";
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);
}
}

View File

@ -0,0 +1,25 @@
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTQueryWithOnCluster.h>
namespace DB
{
class ASTDropNamedCollectionQuery : public IAST, public ASTQueryWithOnCluster
{
public:
std::string collection_name;
bool if_exists = false;
String getID(char) const override { return "DropNamedCollectionQuery"; }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster<ASTDropNamedCollectionQuery>(clone()); }
};
}

View File

@ -0,0 +1,85 @@
#include <Parsers/ASTIdentifier.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ParserAlterNamedCollectionQuery.h>
#include <Parsers/ParserSetQuery.h>
#include <Parsers/ASTAlterNamedCollectionQuery.h>
#include <Parsers/ASTSetQuery.h>
namespace DB
{
bool ParserAlterNamedCollectionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_alter("ALTER");
ParserKeyword s_collection("NAMED COLLECTION");
ParserKeyword s_delete("DELETE");
ParserIdentifier name_p;
ParserSetQuery set_p;
ParserToken s_comma(TokenType::Comma);
String cluster_str;
bool if_exists = false;
ASTPtr collection_name;
ASTPtr set;
std::vector<std::string> delete_keys;
if (!s_alter.ignore(pos, expected))
return false;
if (!s_collection.ignore(pos, expected))
return false;
if (!name_p.parse(pos, collection_name, expected))
return false;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
bool parsed_delete = false;
if (!set_p.parse(pos, set, expected))
{
if (!s_delete.ignore(pos, expected))
return false;
parsed_delete = true;
}
else if (s_delete.ignore(pos, expected))
{
parsed_delete = true;
}
if (parsed_delete)
{
while (true)
{
if (!delete_keys.empty() && !s_comma.ignore(pos))
break;
ASTPtr key;
if (!name_p.parse(pos, key, expected))
return false;
delete_keys.push_back(getIdentifierName(key));
}
}
auto query = std::make_shared<ASTAlterNamedCollectionQuery>();
query->collection_name = getIdentifierName(collection_name);
query->if_exists = if_exists;
query->cluster = std::move(cluster_str);
if (set)
query->changes = set->as<ASTSetQuery>()->changes;
query->delete_keys = delete_keys;
node = query;
return true;
}
}

View File

@ -0,0 +1,14 @@
#pragma once
#include "IParserBase.h"
namespace DB
{
class ParserAlterNamedCollectionQuery : public IParserBase
{
protected:
const char * getName() const override { return "Alter NAMED COLLECTION query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}

View File

@ -9,6 +9,7 @@
#include <Parsers/ASTProjectionDeclaration.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTCreateNamedCollectionQuery.h>
#include <Parsers/ASTTableOverrides.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ParserCreateQuery.h>
@ -1383,6 +1384,59 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
}
bool ParserCreateNamedCollectionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_create("CREATE");
ParserKeyword s_attach("ATTACH");
ParserKeyword s_named_collection("NAMED COLLECTION");
ParserKeyword s_as("AS");
ParserToken s_comma(TokenType::Comma);
ParserIdentifier name_p;
ASTPtr collection_name;
String cluster_str;
if (!s_create.ignore(pos, expected))
return false;
if (!s_named_collection.ignore(pos, expected))
return false;
if (!name_p.parse(pos, collection_name, expected))
return false;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
if (!s_as.ignore(pos, expected))
return false;
SettingsChanges changes;
while (true)
{
if (!changes.empty() && !s_comma.ignore(pos))
break;
changes.push_back(SettingChange{});
if (!ParserSetQuery::parseNameValuePair(changes.back(), pos, expected))
return false;
}
auto query = std::make_shared<ASTCreateNamedCollectionQuery>();
tryGetIdentifierNameInto(collection_name, query->collection_name);
query->changes = changes;
node = query;
return true;
}
bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_create("CREATE");

View File

@ -522,6 +522,13 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserCreateNamedCollectionQuery : public IParserBase
{
protected:
const char * getName() const override { return "CREATE NAMED COLLECTION"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** Query like this:
* CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name

View File

@ -0,0 +1,50 @@
#include <Parsers/ASTIdentifier.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ParserDropNamedCollectionQuery.h>
#include <Parsers/ASTDropNamedCollectionQuery.h>
namespace DB
{
bool ParserDropNamedCollectionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_drop("DROP");
ParserKeyword s_collection("NAMED COLLECTION");
ParserKeyword s_if_exists("IF EXISTS");
ParserIdentifier name_p;
String cluster_str;
bool if_exists = false;
ASTPtr collection_name;
if (!s_drop.ignore(pos, expected))
return false;
if (!s_collection.ignore(pos, expected))
return false;
if (s_if_exists.ignore(pos, expected))
if_exists = true;
if (!name_p.parse(pos, collection_name, expected))
return false;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
auto query = std::make_shared<ASTDropNamedCollectionQuery>();
tryGetIdentifierNameInto(collection_name, query->collection_name);
query->if_exists = if_exists;
query->cluster = std::move(cluster_str);
node = query;
return true;
}
}

View File

@ -0,0 +1,14 @@
#pragma once
#include "IParserBase.h"
namespace DB
{
class ParserDropNamedCollectionQuery : public IParserBase
{
protected:
const char * getName() const override { return "DROP NAMED COLLECTION query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}

View File

@ -5,6 +5,8 @@
#include <Parsers/ParserCreateIndexQuery.h>
#include <Parsers/ParserDropFunctionQuery.h>
#include <Parsers/ParserDropIndexQuery.h>
#include <Parsers/ParserDropNamedCollectionQuery.h>
#include <Parsers/ParserAlterNamedCollectionQuery.h>
#include <Parsers/ParserDropQuery.h>
#include <Parsers/ParserInsertQuery.h>
#include <Parsers/ParserOptimizeQuery.h>
@ -46,6 +48,9 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserCreateSettingsProfileQuery create_settings_profile_p;
ParserCreateFunctionQuery create_function_p;
ParserDropFunctionQuery drop_function_p;
ParserCreateNamedCollectionQuery create_named_collection_p;
ParserDropNamedCollectionQuery drop_named_collection_p;
ParserAlterNamedCollectionQuery alter_named_collection_p;
ParserCreateIndexQuery create_index_p;
ParserDropIndexQuery drop_index_p;
ParserDropAccessEntityQuery drop_access_entity_p;
@ -69,6 +74,9 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|| create_settings_profile_p.parse(pos, node, expected)
|| create_function_p.parse(pos, node, expected)
|| drop_function_p.parse(pos, node, expected)
|| create_named_collection_p.parse(pos, node, expected)
|| drop_named_collection_p.parse(pos, node, expected)
|| alter_named_collection_p.parse(pos, node, expected)
|| create_index_p.parse(pos, node, expected)
|| drop_index_p.parse(pos, node, expected)
|| drop_access_entity_p.parse(pos, node, expected)

View File

@ -572,7 +572,6 @@ ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::stri
std::map<UInt64, std::string> StorageDistributedDirectoryMonitor::getFiles()
{
std::map<UInt64, std::string> files;
size_t new_bytes_count = 0;
fs::directory_iterator end;
for (fs::directory_iterator it{path}; it != end; ++it)
@ -581,23 +580,9 @@ std::map<UInt64, std::string> StorageDistributedDirectoryMonitor::getFiles()
if (!it->is_directory() && startsWith(fs::path(file_path_str).extension(), ".bin"))
{
files[parse<UInt64>(fs::path(file_path_str).stem())] = file_path_str;
new_bytes_count += fs::file_size(fs::path(file_path_str));
}
}
{
std::lock_guard status_lock(status_mutex);
if (status.files_count != files.size())
LOG_TRACE(log, "Files set to {} (was {})", files.size(), status.files_count);
if (status.bytes_count != new_bytes_count)
LOG_TRACE(log, "Bytes set to {} (was {})", new_bytes_count, status.bytes_count);
metric_pending_files.changeTo(files.size());
status.files_count = files.size();
status.bytes_count = new_bytes_count;
}
return files;
}
bool StorageDistributedDirectoryMonitor::processFiles(const std::map<UInt64, std::string> & files)

View File

@ -0,0 +1,174 @@
#include <Storages/NamedCollectionConfiguration.h>
#include <Poco/Util/XMLConfiguration.h>
#include <Common/Exception.h>
#include <Common/SettingsChanges.h>
#include <Common/FieldVisitorToString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int NOT_IMPLEMENTED;
}
namespace NamedCollectionConfiguration
{
template <typename T> T getConfigValue(
const Poco::Util::AbstractConfiguration & config,
const std::string & path)
{
return getConfigValueOrDefault<T>(config, path);
}
template <typename T> T getConfigValueOrDefault(
const Poco::Util::AbstractConfiguration & config,
const std::string & path,
const T * default_value)
{
if (!config.has(path))
{
if (!default_value)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", path);
return *default_value;
}
if constexpr (std::is_same_v<T, String>)
return config.getString(path);
else if constexpr (std::is_same_v<T, UInt64>)
return config.getUInt64(path);
else if constexpr (std::is_same_v<T, Int64>)
return config.getInt64(path);
else if constexpr (std::is_same_v<T, Float64>)
return config.getDouble(path);
else
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Unsupported type in getConfigValueOrDefault(). "
"Supported types are String, UInt64, Int64, Float64");
}
template<typename T> void setConfigValue(
Poco::Util::AbstractConfiguration & config,
const std::string & path,
const T & value,
bool update)
{
if (!update && config.has(path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key `{}` already exists", path);
if constexpr (std::is_same_v<T, String>)
config.setString(path, value);
else if constexpr (std::is_same_v<T, UInt64>)
config.setUInt64(path, value);
else if constexpr (std::is_same_v<T, Int64>)
config.setInt64(path, value);
else if constexpr (std::is_same_v<T, Float64>)
config.setDouble(path, value);
else
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Unsupported type in setConfigValue(). "
"Supported types are String, UInt64, Int64, Float64");
}
template <typename T> void copyConfigValue(
const Poco::Util::AbstractConfiguration & from_config,
const std::string & from_path,
Poco::Util::AbstractConfiguration & to_config,
const std::string & to_path)
{
if (!from_config.has(from_path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", from_path);
if (to_config.has(to_path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key `{}` already exists", to_path);
if constexpr (std::is_same_v<T, String>)
to_config.setString(to_path, from_config.getString(from_path));
else if constexpr (std::is_same_v<T, UInt64>)
to_config.setUInt64(to_path, from_config.getUInt64(from_path));
else if constexpr (std::is_same_v<T, Int64>)
to_config.setInt64(to_path, from_config.getInt64(from_path));
else if constexpr (std::is_same_v<T, Float64>)
to_config.setDouble(to_path, from_config.getDouble(from_path));
else
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Unsupported type in copyConfigValue(). "
"Supported types are String, UInt64, Int64, Float64");
}
void removeConfigValue(
Poco::Util::AbstractConfiguration & config,
const std::string & path)
{
if (!config.has(path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", path);
config.remove(path);
}
ConfigurationPtr createEmptyConfiguration(const std::string & root_name)
{
using DocumentPtr = Poco::AutoPtr<Poco::XML::Document>;
using ElementPtr = Poco::AutoPtr<Poco::XML::Element>;
DocumentPtr xml_document(new Poco::XML::Document());
ElementPtr root_element(xml_document->createElement(root_name));
xml_document->appendChild(root_element);
ConfigurationPtr config(new Poco::Util::XMLConfiguration(xml_document));
return config;
}
ConfigurationPtr createConfiguration(const std::string & root_name, const SettingsChanges & settings)
{
namespace Configuration = NamedCollectionConfiguration;
auto config = Configuration::createEmptyConfiguration(root_name);
for (const auto & [name, value] : settings)
Configuration::setConfigValue<String>(*config, name, convertFieldToString(value));
return config;
}
template String getConfigValue<String>(const Poco::Util::AbstractConfiguration & config,
const std::string & path);
template UInt64 getConfigValue<UInt64>(const Poco::Util::AbstractConfiguration & config,
const std::string & path);
template Int64 getConfigValue<Int64>(const Poco::Util::AbstractConfiguration & config,
const std::string & path);
template Float64 getConfigValue<Float64>(const Poco::Util::AbstractConfiguration & config,
const std::string & path);
template String getConfigValueOrDefault<String>(const Poco::Util::AbstractConfiguration & config,
const std::string & path, const String * default_value);
template UInt64 getConfigValueOrDefault<UInt64>(const Poco::Util::AbstractConfiguration & config,
const std::string & path, const UInt64 * default_value);
template Int64 getConfigValueOrDefault<Int64>(const Poco::Util::AbstractConfiguration & config,
const std::string & path, const Int64 * default_value);
template Float64 getConfigValueOrDefault<Float64>(const Poco::Util::AbstractConfiguration & config,
const std::string & path, const Float64 * default_value);
template void setConfigValue<String>(Poco::Util::AbstractConfiguration & config,
const std::string & path, const String & value, bool update);
template void setConfigValue<UInt64>(Poco::Util::AbstractConfiguration & config,
const std::string & path, const UInt64 & value, bool update);
template void setConfigValue<Int64>(Poco::Util::AbstractConfiguration & config,
const std::string & path, const Int64 & value, bool update);
template void setConfigValue<Float64>(Poco::Util::AbstractConfiguration & config,
const std::string & path, const Float64 & value, bool update);
template void copyConfigValue<String>(const Poco::Util::AbstractConfiguration & from_config, const std::string & from_path,
Poco::Util::AbstractConfiguration & to_config, const std::string & to_path);
template void copyConfigValue<UInt64>(const Poco::Util::AbstractConfiguration & from_config, const std::string & from_path,
Poco::Util::AbstractConfiguration & to_config, const std::string & to_path);
template void copyConfigValue<Int64>(const Poco::Util::AbstractConfiguration & from_config, const std::string & from_path,
Poco::Util::AbstractConfiguration & to_config, const std::string & to_path);
template void copyConfigValue<Float64>(const Poco::Util::AbstractConfiguration & from_config, const std::string & from_path,
Poco::Util::AbstractConfiguration & to_config, const std::string & to_path);
}
}

View File

@ -0,0 +1,44 @@
#pragma once
#include <Poco/Util/AbstractConfiguration.h>
namespace DB
{
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
class SettingsChanges;
namespace NamedCollectionConfiguration
{
ConfigurationPtr createEmptyConfiguration(const std::string & root_name);
template <typename T> T getConfigValue(
const Poco::Util::AbstractConfiguration & config,
const std::string & path);
template <typename T> T getConfigValueOrDefault(
const Poco::Util::AbstractConfiguration & config,
const std::string & path,
const T * default_value = nullptr);
template<typename T> void setConfigValue(
Poco::Util::AbstractConfiguration & config,
const std::string & path,
const T & value,
bool update = false);
template <typename T> void copyConfigValue(
const Poco::Util::AbstractConfiguration & from_config,
const std::string & from_path,
Poco::Util::AbstractConfiguration & to_config,
const std::string & to_path);
void removeConfigValue(
Poco::Util::AbstractConfiguration & config,
const std::string & path);
ConfigurationPtr createConfiguration(const std::string & root_name, const SettingsChanges & settings);
}
}

View File

@ -0,0 +1,434 @@
#include <Storages/NamedCollectionUtils.h>
#include <Common/escapeForFileName.h>
#include <Common/FieldVisitorToString.h>
#include <Common/logger_useful.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/WriteHelpers.h>
#include <Parsers/formatAST.h>
#include <Parsers/ASTCreateNamedCollectionQuery.h>
#include <Parsers/ASTAlterNamedCollectionQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Interpreters/Context.h>
#include <Storages/NamedCollections.h>
#include <Storages/NamedCollectionConfiguration.h>
namespace fs = std::filesystem;
namespace DB
{
namespace ErrorCodes
{
extern const int NAMED_COLLECTION_ALREADY_EXISTS;
extern const int NAMED_COLLECTION_DOESNT_EXIST;
extern const int BAD_ARGUMENTS;
}
namespace NamedCollectionUtils
{
class LoadFromConfig
{
private:
const Poco::Util::AbstractConfiguration & config;
public:
explicit LoadFromConfig(const Poco::Util::AbstractConfiguration & config_)
: config(config_) {}
std::vector<std::string> listCollections() const
{
Poco::Util::AbstractConfiguration::Keys collections_names;
config.keys(NAMED_COLLECTIONS_CONFIG_PREFIX, collections_names);
return collections_names;
}
NamedCollectionsMap getAll() const
{
NamedCollectionsMap result;
for (const auto & collection_name : listCollections())
{
if (result.contains(collection_name))
{
throw Exception(
ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS,
"Found duplicate named collection `{}`",
collection_name);
}
result.emplace(collection_name, get(collection_name));
}
return result;
}
MutableNamedCollectionPtr get(const std::string & collection_name) const
{
const auto collection_prefix = getCollectionPrefix(collection_name);
std::queue<std::string> enumerate_input;
std::set<std::string> enumerate_result;
enumerate_input.push(collection_prefix);
collectKeys(config, std::move(enumerate_input), enumerate_result);
/// Collection does not have any keys.
/// (`enumerate_result` == <collection_path>).
const bool collection_is_empty = enumerate_result.size() == 1
&& *enumerate_result.begin() == collection_prefix;
std::set<std::string> keys;
if (!collection_is_empty)
{
/// Skip collection prefix and add +1 to avoid '.' in the beginning.
for (const auto & path : enumerate_result)
keys.emplace(path.substr(collection_prefix.size() + 1));
}
return NamedCollection::create(
config, collection_name, collection_prefix, keys, SourceId::CONFIG, /* is_mutable */false);
}
private:
static constexpr auto NAMED_COLLECTIONS_CONFIG_PREFIX = "named_collections";
static std::string getCollectionPrefix(const std::string & collection_name)
{
return fmt::format("{}.{}", NAMED_COLLECTIONS_CONFIG_PREFIX, collection_name);
}
/// Enumerate keys paths of the config recursively.
/// E.g. if `enumerate_paths` = {"root.key1"} and config like
/// <root>
/// <key0></key0>
/// <key1>
/// <key2></key2>
/// <key3>
/// <key4></key4>
/// </key3>
/// </key1>
/// </root>
/// the `result` will contain two strings: "root.key1.key2" and "root.key1.key3.key4"
static void collectKeys(
const Poco::Util::AbstractConfiguration & config,
std::queue<std::string> enumerate_paths,
std::set<std::string> & result)
{
if (enumerate_paths.empty())
return;
auto initial_paths = std::move(enumerate_paths);
enumerate_paths = {};
while (!initial_paths.empty())
{
auto path = initial_paths.front();
initial_paths.pop();
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(path, keys);
if (keys.empty())
{
result.insert(path);
}
else
{
for (const auto & key : keys)
enumerate_paths.emplace(path + '.' + key);
}
}
collectKeys(config, enumerate_paths, result);
}
};
class LoadFromSQL : private WithContext
{
private:
const std::string metadata_path;
public:
explicit LoadFromSQL(ContextPtr context_)
: WithContext(context_)
, metadata_path(
fs::canonical(context_->getPath()) / NAMED_COLLECTIONS_METADATA_DIRECTORY)
{
if (fs::exists(metadata_path))
cleanUp();
else
fs::create_directories(metadata_path);
}
std::vector<std::string> listCollections() const
{
std::vector<std::string> collection_names;
fs::directory_iterator it{metadata_path};
for (; it != fs::directory_iterator{}; ++it)
{
const auto & current_path = it->path();
if (current_path.extension() == ".sql")
{
collection_names.push_back(it->path().stem());
}
else
{
LOG_WARNING(
&Poco::Logger::get("NamedCollectionsLoadFromSQL"),
"Unexpected file {} in named collections directory",
current_path.filename().string());
}
}
return collection_names;
}
NamedCollectionsMap getAll() const
{
NamedCollectionsMap result;
for (const auto & collection_name : listCollections())
{
if (result.contains(collection_name))
{
throw Exception(
ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS,
"Found duplicate named collection `{}`",
collection_name);
}
result.emplace(collection_name, get(collection_name));
}
return result;
}
MutableNamedCollectionPtr get(const std::string & collection_name) const
{
const auto query = readCreateQueryFromMetadata(
getMetadataPath(collection_name),
getContext()->getSettingsRef());
return createNamedCollectionFromAST(query);
}
MutableNamedCollectionPtr create(const ASTCreateNamedCollectionQuery & query)
{
writeCreateQueryToMetadata(
query,
getMetadataPath(query.collection_name),
getContext()->getSettingsRef());
return createNamedCollectionFromAST(query);
}
void update(const ASTAlterNamedCollectionQuery & query)
{
const auto path = getMetadataPath(query.collection_name);
auto create_query = readCreateQueryFromMetadata(path, getContext()->getSettings());
std::unordered_map<std::string, Field> result_changes_map;
for (const auto & [name, value] : query.changes)
{
auto [it, inserted] = result_changes_map.emplace(name, value);
if (!inserted)
{
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Value with key `{}` is used twice in the SET query",
name, query.collection_name);
}
}
for (const auto & [name, value] : create_query.changes)
result_changes_map.emplace(name, value);
for (const auto & delete_key : query.delete_keys)
{
auto it = result_changes_map.find(delete_key);
if (it == result_changes_map.end())
{
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Cannot delete key `{}` because it does not exist in collection",
delete_key);
}
else
result_changes_map.erase(it);
}
create_query.changes.clear();
for (const auto & [name, value] : result_changes_map)
create_query.changes.emplace_back(name, value);
writeCreateQueryToMetadata(
create_query,
getMetadataPath(query.collection_name),
getContext()->getSettingsRef(),
true);
}
void remove(const std::string & collection_name)
{
if (!removeIfExists(collection_name))
{
throw Exception(
ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST,
"Cannot remove collection `{}`, because it doesn't exist",
collection_name);
}
}
bool removeIfExists(const std::string & collection_name)
{
auto collection_path = getMetadataPath(collection_name);
if (fs::exists(collection_path))
{
fs::remove(collection_path);
return true;
}
return false;
}
private:
static constexpr auto NAMED_COLLECTIONS_METADATA_DIRECTORY = "named_collections";
static MutableNamedCollectionPtr createNamedCollectionFromAST(
const ASTCreateNamedCollectionQuery & query)
{
const auto & collection_name = query.collection_name;
const auto config = NamedCollectionConfiguration::createConfiguration(
collection_name, query.changes);
std::set<std::string> keys;
for (const auto & [name, _] : query.changes)
keys.insert(name);
return NamedCollection::create(
*config, collection_name, "", keys, SourceId::SQL, /* is_mutable */true);
}
std::string getMetadataPath(const std::string & collection_name) const
{
return fs::path(metadata_path) / (escapeForFileName(collection_name) + ".sql");
}
/// Delete .tmp files. They could be left undeleted in case of
/// some exception or abrupt server restart.
void cleanUp()
{
fs::directory_iterator it{metadata_path};
std::vector<std::string> files_to_remove;
for (; it != fs::directory_iterator{}; ++it)
{
const auto & current_path = it->path();
if (current_path.extension() == ".tmp")
files_to_remove.push_back(current_path);
}
for (const auto & file : files_to_remove)
fs::remove(file);
}
static ASTCreateNamedCollectionQuery readCreateQueryFromMetadata(
const std::string & path,
const Settings & settings)
{
ReadBufferFromFile in(path);
std::string query;
readStringUntilEOF(query, in);
ParserCreateNamedCollectionQuery parser;
auto ast = parseQuery(parser, query, "in file " + path, 0, settings.max_parser_depth);
const auto & create_query = ast->as<const ASTCreateNamedCollectionQuery &>();
return create_query;
}
static void writeCreateQueryToMetadata(
const ASTCreateNamedCollectionQuery & query,
const std::string & path,
const Settings & settings,
bool replace = false)
{
if (!replace && fs::exists(path))
{
throw Exception(
ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS,
"Metadata file {} for named collection already exists",
path);
}
auto tmp_path = path + ".tmp";
String formatted_query = serializeAST(query);
WriteBufferFromFile out(tmp_path, formatted_query.size(), O_WRONLY | O_CREAT | O_EXCL);
writeString(formatted_query, out);
out.next();
if (settings.fsync_metadata)
out.sync();
out.close();
fs::rename(tmp_path, path);
}
};
std::unique_lock<std::mutex> lockNamedCollectionsTransaction()
{
static std::mutex transaction_lock;
return std::unique_lock(transaction_lock);
}
void loadFromConfig(const Poco::Util::AbstractConfiguration & config)
{
auto lock = lockNamedCollectionsTransaction();
NamedCollectionFactory::instance().add(LoadFromConfig(config).getAll());
}
void reloadFromConfig(const Poco::Util::AbstractConfiguration & config)
{
auto lock = lockNamedCollectionsTransaction();
auto collections = LoadFromConfig(config).getAll();
auto & instance = NamedCollectionFactory::instance();
instance.removeById(SourceId::CONFIG);
instance.add(collections);
}
void loadFromSQL(ContextPtr context)
{
auto lock = lockNamedCollectionsTransaction();
NamedCollectionFactory::instance().add(LoadFromSQL(context).getAll());
}
void removeFromSQL(const std::string & collection_name, ContextPtr context)
{
auto lock = lockNamedCollectionsTransaction();
LoadFromSQL(context).remove(collection_name);
NamedCollectionFactory::instance().remove(collection_name);
}
void removeIfExistsFromSQL(const std::string & collection_name, ContextPtr context)
{
auto lock = lockNamedCollectionsTransaction();
LoadFromSQL(context).removeIfExists(collection_name);
NamedCollectionFactory::instance().removeIfExists(collection_name);
}
void createFromSQL(const ASTCreateNamedCollectionQuery & query, ContextPtr context)
{
auto lock = lockNamedCollectionsTransaction();
NamedCollectionFactory::instance().add(query.collection_name, LoadFromSQL(context).create(query));
}
void updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr context)
{
auto lock = lockNamedCollectionsTransaction();
LoadFromSQL(context).update(query);
auto collection = NamedCollectionFactory::instance().getMutable(query.collection_name);
auto collection_lock = collection->lock();
for (const auto & [name, value] : query.changes)
collection->setOrUpdate<String, true>(name, convertFieldToString(value));
for (const auto & key : query.delete_keys)
collection->remove<true>(key);
}
}
}

View File

@ -0,0 +1,40 @@
#pragma once
#include <Interpreters/Context_fwd.h>
namespace Poco { namespace Util { class AbstractConfiguration; } }
namespace DB
{
class ASTCreateNamedCollectionQuery;
class ASTAlterNamedCollectionQuery;
namespace NamedCollectionUtils
{
enum class SourceId
{
NONE = 0,
CONFIG = 1,
SQL = 2,
};
void loadFromConfig(const Poco::Util::AbstractConfiguration & config);
void reloadFromConfig(const Poco::Util::AbstractConfiguration & config);
/// Load named collections from `context->getPath() / named_collections /`.
void loadFromSQL(ContextPtr context);
/// Remove collection as well as its metadata from `context->getPath() / named_collections /`.
void removeFromSQL(const std::string & collection_name, ContextPtr context);
void removeIfExistsFromSQL(const std::string & collection_name, ContextPtr context);
/// Create a new collection from AST and put it to `context->getPath() / named_collections /`.
void createFromSQL(const ASTCreateNamedCollectionQuery & query, ContextPtr context);
/// Update definition of already existing collection from AST and update result in `context->getPath() / named_collections /`.
void updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr context);
}
}

View File

@ -1,17 +1,11 @@
#include "NamedCollections.h"
#include <base/find_symbols.h>
#include <Common/assert_cast.h>
#include <Common/FieldVisitorToString.h>
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/Util/XMLConfiguration.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <Storages/NamedCollectionConfiguration.h>
#include <Storages/NamedCollectionUtils.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <ranges>
@ -20,66 +14,13 @@ namespace DB
namespace ErrorCodes
{
extern const int UNKNOWN_NAMED_COLLECTION;
extern const int NAMED_COLLECTION_DOESNT_EXIST;
extern const int NAMED_COLLECTION_ALREADY_EXISTS;
extern const int BAD_ARGUMENTS;
extern const int NOT_IMPLEMENTED;
extern const int LOGICAL_ERROR;
extern const int NAMED_COLLECTION_IS_IMMUTABLE;
}
namespace
{
constexpr auto NAMED_COLLECTIONS_CONFIG_PREFIX = "named_collections";
namespace Configuration = NamedCollectionConfiguration;
std::string getCollectionPrefix(const std::string & collection_name)
{
return fmt::format("{}.{}", NAMED_COLLECTIONS_CONFIG_PREFIX, collection_name);
}
/// Enumerate keys paths of the config recursively.
/// E.g. if `enumerate_paths` = {"root.key1"} and config like
/// <root>
/// <key0></key0>
/// <key1>
/// <key2></key2>
/// <key3>
/// <key4></key4>
/// </key3>
/// </key1>
/// </root>
/// the `result` will contain two strings: "root.key1.key2" and "root.key1.key3.key4"
void collectKeys(
const Poco::Util::AbstractConfiguration & config,
std::queue<std::string> enumerate_paths,
std::set<std::string> & result)
{
if (enumerate_paths.empty())
return;
auto initial_paths = std::move(enumerate_paths);
enumerate_paths = {};
while (!initial_paths.empty())
{
auto path = initial_paths.front();
initial_paths.pop();
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(path, keys);
if (keys.empty())
{
result.insert(path);
}
else
{
for (const auto & key : keys)
enumerate_paths.emplace(path + '.' + key);
}
}
collectKeys(config, enumerate_paths, result);
}
}
NamedCollectionFactory & NamedCollectionFactory::instance()
{
@ -87,38 +28,6 @@ NamedCollectionFactory & NamedCollectionFactory::instance()
return instance;
}
void NamedCollectionFactory::initialize(const Poco::Util::AbstractConfiguration & config_)
{
std::lock_guard lock(mutex);
if (is_initialized)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Named collection factory already initialized");
}
config = &config_;
is_initialized = true;
}
void NamedCollectionFactory::reload(const Poco::Util::AbstractConfiguration & config_)
{
std::lock_guard lock(mutex);
config = &config_;
loaded_named_collections.clear();
}
void NamedCollectionFactory::assertInitialized(
std::lock_guard<std::mutex> & /* lock */) const
{
if (!is_initialized)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Named collection factory must be initialized before being used");
}
}
bool NamedCollectionFactory::exists(const std::string & collection_name) const
{
std::lock_guard lock(mutex);
@ -127,62 +36,84 @@ bool NamedCollectionFactory::exists(const std::string & collection_name) const
bool NamedCollectionFactory::existsUnlocked(
const std::string & collection_name,
std::lock_guard<std::mutex> & lock) const
std::lock_guard<std::mutex> & /* lock */) const
{
assertInitialized(lock);
/// Named collections can be added via SQL command or via config.
/// Named collections from config are loaded on first access,
/// therefore it might not be in `named_collections` map yet.
return loaded_named_collections.contains(collection_name)
|| config->has(getCollectionPrefix(collection_name));
return loaded_named_collections.contains(collection_name);
}
NamedCollectionPtr NamedCollectionFactory::get(const std::string & collection_name) const
{
std::lock_guard lock(mutex);
assertInitialized(lock);
if (!existsUnlocked(collection_name, lock))
auto collection = tryGetUnlocked(collection_name, lock);
if (!collection)
{
throw Exception(
ErrorCodes::UNKNOWN_NAMED_COLLECTION,
ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST,
"There is no named collection `{}`",
collection_name);
}
return getImpl(collection_name, lock);
return collection;
}
NamedCollectionPtr NamedCollectionFactory::tryGet(const std::string & collection_name) const
{
std::lock_guard lock(mutex);
assertInitialized(lock);
if (!existsUnlocked(collection_name, lock))
return nullptr;
return getImpl(collection_name, lock);
return tryGetUnlocked(collection_name, lock);
}
NamedCollectionPtr NamedCollectionFactory::getImpl(
MutableNamedCollectionPtr NamedCollectionFactory::getMutable(
const std::string & collection_name) const
{
std::lock_guard lock(mutex);
auto collection = tryGetUnlocked(collection_name, lock);
if (!collection)
{
throw Exception(
ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST,
"There is no named collection `{}`",
collection_name);
}
else if (!collection->isMutable())
{
throw Exception(
ErrorCodes::NAMED_COLLECTION_IS_IMMUTABLE,
"Cannot get collection `{}` for modification, "
"because collection was defined as immutable",
collection_name);
}
return collection;
}
MutableNamedCollectionPtr NamedCollectionFactory::tryGetUnlocked(
const std::string & collection_name,
std::lock_guard<std::mutex> & /* lock */) const
{
auto it = loaded_named_collections.find(collection_name);
if (it == loaded_named_collections.end())
{
it = loaded_named_collections.emplace(
collection_name,
NamedCollection::create(*config, collection_name)).first;
}
return nullptr;
return it->second;
}
void NamedCollectionFactory::add(
const std::string & collection_name,
NamedCollectionPtr collection)
MutableNamedCollectionPtr collection)
{
std::lock_guard lock(mutex);
return addUnlocked(collection_name, collection, lock);
}
void NamedCollectionFactory::add(NamedCollectionsMap collections)
{
std::lock_guard lock(mutex);
for (const auto & [collection_name, collection] : collections)
addUnlocked(collection_name, collection, lock);
}
void NamedCollectionFactory::addUnlocked(
const std::string & collection_name,
MutableNamedCollectionPtr collection,
std::lock_guard<std::mutex> & /* lock */)
{
auto [it, inserted] = loaded_named_collections.emplace(collection_name, collection);
if (!inserted)
{
@ -196,93 +127,104 @@ void NamedCollectionFactory::add(
void NamedCollectionFactory::remove(const std::string & collection_name)
{
std::lock_guard lock(mutex);
assertInitialized(lock);
if (!existsUnlocked(collection_name, lock))
bool removed = removeIfExistsUnlocked(collection_name, lock);
if (!removed)
{
throw Exception(
ErrorCodes::UNKNOWN_NAMED_COLLECTION,
ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST,
"There is no named collection `{}`",
collection_name);
}
if (config->has(collection_name))
{
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Collection {} is defined in config and cannot be removed",
collection_name);
}
[[maybe_unused]] auto removed = loaded_named_collections.erase(collection_name);
assert(removed);
}
NamedCollectionFactory::NamedCollections NamedCollectionFactory::getAll() const
void NamedCollectionFactory::removeIfExists(const std::string & collection_name)
{
std::lock_guard lock(mutex);
assertInitialized(lock);
removeIfExistsUnlocked(collection_name, lock);
}
NamedCollections result(loaded_named_collections);
bool NamedCollectionFactory::removeIfExistsUnlocked(
const std::string & collection_name,
std::lock_guard<std::mutex> & lock)
{
auto collection = tryGetUnlocked(collection_name, lock);
if (!collection)
return false;
Poco::Util::AbstractConfiguration::Keys config_collections_names;
config->keys(NAMED_COLLECTIONS_CONFIG_PREFIX, config_collections_names);
for (const auto & collection_name : config_collections_names)
if (!collection->isMutable())
{
if (result.contains(collection_name))
continue;
result.emplace(collection_name, NamedCollection::create(*config, collection_name));
throw Exception(
ErrorCodes::NAMED_COLLECTION_IS_IMMUTABLE,
"Cannot get collection `{}` for modification, "
"because collection was defined as immutable",
collection_name);
}
loaded_named_collections.erase(collection_name);
return true;
}
return result;
void NamedCollectionFactory::removeById(NamedCollectionUtils::SourceId id)
{
std::lock_guard lock(mutex);
std::erase_if(
loaded_named_collections,
[&](const auto & value) { return value.second->getSourceId() == id; });
}
NamedCollectionsMap NamedCollectionFactory::getAll() const
{
std::lock_guard lock(mutex);
return loaded_named_collections;
}
class NamedCollection::Impl
{
private:
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
/// Named collection configuration
/// <collection1>
/// ...
/// </collection1>
ConfigurationPtr config;
Keys keys;
Impl(ConfigurationPtr config_, const Keys & keys_) : config(config_) , keys(keys_) {}
public:
Impl(const Poco::Util::AbstractConfiguration & config_,
const std::string & collection_name_,
const Keys & keys_)
: config(createEmptyConfiguration(collection_name_))
, keys(keys_)
static ImplPtr create(
const Poco::Util::AbstractConfiguration & config,
const std::string & collection_name,
const std::string & collection_path,
const Keys & keys)
{
auto collection_path = getCollectionPrefix(collection_name_);
auto collection_config = NamedCollectionConfiguration::createEmptyConfiguration(collection_name);
for (const auto & key : keys)
copyConfigValue<String>(config_, collection_path + '.' + key, *config, key);
Configuration::copyConfigValue<String>(
config, collection_path + '.' + key, *collection_config, key);
return std::unique_ptr<Impl>(new Impl(collection_config, keys));
}
template <typename T> T get(const Key & key) const
{
return getConfigValue<T>(*config, key);
return Configuration::getConfigValue<T>(*config, key);
}
template <typename T> T getOrDefault(const Key & key, const T & default_value) const
{
return getConfigValueOrDefault<T>(*config, key, &default_value);
return Configuration::getConfigValueOrDefault<T>(*config, key, &default_value);
}
template <typename T> void set(const Key & key, const T & value, bool update_if_exists)
{
setConfigValue<T>(*config, key, value, update_if_exists);
Configuration::setConfigValue<T>(*config, key, value, update_if_exists);
if (!keys.contains(key))
keys.insert(key);
}
ImplPtr createCopy(const std::string & collection_name_) const
{
return create(*config, collection_name_, "", keys);
}
void remove(const Key & key)
{
removeConfigValue(*config, key);
Configuration::removeConfigValue(*config, key);
[[maybe_unused]] auto removed = keys.erase(key);
assert(removed);
}
@ -292,11 +234,6 @@ public:
return keys;
}
ImplPtr copy() const
{
return std::make_unique<Impl>(*this);
}
std::string dumpStructure() const
{
/// Convert a collection config like
@ -347,186 +284,108 @@ public:
}
return wb.str();
}
private:
template <typename T> static T getConfigValue(
const Poco::Util::AbstractConfiguration & config,
const std::string & path)
{
return getConfigValueOrDefault<T>(config, path);
}
template <typename T> static T getConfigValueOrDefault(
const Poco::Util::AbstractConfiguration & config,
const std::string & path,
const T * default_value = nullptr)
{
if (!config.has(path))
{
if (!default_value)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", path);
return *default_value;
}
if constexpr (std::is_same_v<T, String>)
return config.getString(path);
else if constexpr (std::is_same_v<T, UInt64>)
return config.getUInt64(path);
else if constexpr (std::is_same_v<T, Int64>)
return config.getInt64(path);
else if constexpr (std::is_same_v<T, Float64>)
return config.getDouble(path);
else
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Unsupported type in getConfigValueOrDefault(). "
"Supported types are String, UInt64, Int64, Float64");
}
template<typename T> static void setConfigValue(
Poco::Util::AbstractConfiguration & config,
const std::string & path,
const T & value,
bool update = false)
{
if (!update && config.has(path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key `{}` already exists", path);
if constexpr (std::is_same_v<T, String>)
config.setString(path, value);
else if constexpr (std::is_same_v<T, UInt64>)
config.setUInt64(path, value);
else if constexpr (std::is_same_v<T, Int64>)
config.setInt64(path, value);
else if constexpr (std::is_same_v<T, Float64>)
config.setDouble(path, value);
else
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Unsupported type in setConfigValue(). "
"Supported types are String, UInt64, Int64, Float64");
}
template <typename T> static void copyConfigValue(
const Poco::Util::AbstractConfiguration & from_config,
const std::string & from_path,
Poco::Util::AbstractConfiguration & to_config,
const std::string & to_path)
{
if (!from_config.has(from_path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", from_path);
if (to_config.has(to_path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key `{}` already exists", to_path);
if constexpr (std::is_same_v<T, String>)
to_config.setString(to_path, from_config.getString(from_path));
else if constexpr (std::is_same_v<T, std::string>)
to_config.setString(to_path, from_config.getString(from_path));
else if constexpr (std::is_same_v<T, UInt64>)
to_config.setUInt64(to_path, from_config.getUInt64(from_path));
else if constexpr (std::is_same_v<T, Int64>)
to_config.setInt64(to_path, from_config.getInt64(from_path));
else if constexpr (std::is_same_v<T, Float64>)
to_config.setDouble(to_path, from_config.getDouble(from_path));
else
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Unsupported type in copyConfigValue(). "
"Supported types are String, UInt64, Int64, Float64");
}
static void removeConfigValue(
Poco::Util::AbstractConfiguration & config,
const std::string & path)
{
if (!config.has(path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", path);
config.remove(path);
}
static ConfigurationPtr createEmptyConfiguration(const std::string & root_name)
{
using DocumentPtr = Poco::AutoPtr<Poco::XML::Document>;
DocumentPtr xml_document(new Poco::XML::Document());
xml_document->appendChild(xml_document->createElement(root_name));
ConfigurationPtr config(new Poco::Util::XMLConfiguration(xml_document));
return config;
}
};
NamedCollection::NamedCollection(
const Poco::Util::AbstractConfiguration & config,
const std::string & collection_path,
const Keys & keys)
: NamedCollection(std::make_unique<Impl>(config, collection_path, keys))
{
}
NamedCollection::NamedCollection(ImplPtr pimpl_)
ImplPtr pimpl_,
const std::string & collection_name_,
SourceId source_id_,
bool is_mutable_)
: pimpl(std::move(pimpl_))
, collection_name(collection_name_)
, source_id(source_id_)
, is_mutable(is_mutable_)
{
}
NamedCollectionPtr NamedCollection::create(
MutableNamedCollectionPtr NamedCollection::create(
const Poco::Util::AbstractConfiguration & config,
const std::string & collection_name)
const std::string & collection_name,
const std::string & collection_path,
const Keys & keys,
SourceId source_id,
bool is_mutable)
{
const auto collection_prefix = getCollectionPrefix(collection_name);
std::queue<std::string> enumerate_input;
std::set<std::string> enumerate_result;
enumerate_input.push(collection_prefix);
collectKeys(config, std::move(enumerate_input), enumerate_result);
/// Collection does not have any keys.
/// (`enumerate_result` == <collection_path>).
const bool collection_is_empty = enumerate_result.size() == 1;
std::set<std::string> keys;
if (!collection_is_empty)
{
/// Skip collection prefix and add +1 to avoid '.' in the beginning.
for (const auto & path : enumerate_result)
keys.emplace(path.substr(collection_prefix.size() + 1));
}
return std::make_unique<NamedCollection>(config, collection_name, keys);
auto impl = Impl::create(config, collection_name, collection_path, keys);
return std::unique_ptr<NamedCollection>(
new NamedCollection(std::move(impl), collection_name, source_id, is_mutable));
}
template <typename T> T NamedCollection::get(const Key & key) const
{
std::lock_guard lock(mutex);
return pimpl->get<T>(key);
}
template <typename T> T NamedCollection::getOrDefault(const Key & key, const T & default_value) const
{
std::lock_guard lock(mutex);
return pimpl->getOrDefault<T>(key, default_value);
}
template <typename T> void NamedCollection::set(const Key & key, const T & value, bool update_if_exists)
template <typename T, bool Locked> void NamedCollection::set(const Key & key, const T & value)
{
pimpl->set<T>(key, value, update_if_exists);
assertMutable();
std::unique_lock lock(mutex, std::defer_lock);
if constexpr (!Locked)
lock.lock();
pimpl->set<T>(key, value, false);
}
void NamedCollection::remove(const Key & key)
template <typename T, bool Locked> void NamedCollection::setOrUpdate(const Key & key, const T & value)
{
assertMutable();
std::unique_lock lock(mutex, std::defer_lock);
if constexpr (!Locked)
lock.lock();
pimpl->set<T>(key, value, true);
}
template <bool Locked> void NamedCollection::remove(const Key & key)
{
assertMutable();
std::unique_lock lock(mutex, std::defer_lock);
if constexpr (!Locked)
lock.lock();
pimpl->remove(key);
}
std::shared_ptr<NamedCollection> NamedCollection::duplicate() const
void NamedCollection::assertMutable() const
{
return std::make_shared<NamedCollection>(pimpl->copy());
if (!is_mutable)
{
throw Exception(
ErrorCodes::NAMED_COLLECTION_IS_IMMUTABLE,
"Cannot change named collection because it is immutable");
}
}
MutableNamedCollectionPtr NamedCollection::duplicate() const
{
std::lock_guard lock(mutex);
auto impl = pimpl->createCopy(collection_name);
return std::unique_ptr<NamedCollection>(
new NamedCollection(
std::move(impl), collection_name, NamedCollectionUtils::SourceId::NONE, true));
}
NamedCollection::Keys NamedCollection::getKeys() const
{
std::lock_guard lock(mutex);
return pimpl->getKeys();
}
std::string NamedCollection::dumpStructure() const
{
std::lock_guard lock(mutex);
return pimpl->dumpStructure();
}
std::unique_lock<std::mutex> NamedCollection::lock()
{
return std::unique_lock(mutex);
}
template String NamedCollection::get<String>(const NamedCollection::Key & key) const;
template UInt64 NamedCollection::get<UInt64>(const NamedCollection::Key & key) const;
template Int64 NamedCollection::get<Int64>(const NamedCollection::Key & key) const;
@ -537,9 +396,25 @@ template UInt64 NamedCollection::getOrDefault<UInt64>(const NamedCollection::Key
template Int64 NamedCollection::getOrDefault<Int64>(const NamedCollection::Key & key, const Int64 & default_value) const;
template Float64 NamedCollection::getOrDefault<Float64>(const NamedCollection::Key & key, const Float64 & default_value) const;
template void NamedCollection::set<String>(const NamedCollection::Key & key, const String & value, bool update_if_exists);
template void NamedCollection::set<UInt64>(const NamedCollection::Key & key, const UInt64 & value, bool update_if_exists);
template void NamedCollection::set<Int64>(const NamedCollection::Key & key, const Int64 & value, bool update_if_exists);
template void NamedCollection::set<Float64>(const NamedCollection::Key & key, const Float64 & value, bool update_if_exists);
template void NamedCollection::set<String, true>(const NamedCollection::Key & key, const String & value);
template void NamedCollection::set<String, false>(const NamedCollection::Key & key, const String & value);
template void NamedCollection::set<UInt64, true>(const NamedCollection::Key & key, const UInt64 & value);
template void NamedCollection::set<UInt64, false>(const NamedCollection::Key & key, const UInt64 & value);
template void NamedCollection::set<Int64, true>(const NamedCollection::Key & key, const Int64 & value);
template void NamedCollection::set<Int64, false>(const NamedCollection::Key & key, const Int64 & value);
template void NamedCollection::set<Float64, true>(const NamedCollection::Key & key, const Float64 & value);
template void NamedCollection::set<Float64, false>(const NamedCollection::Key & key, const Float64 & value);
template void NamedCollection::setOrUpdate<String, true>(const NamedCollection::Key & key, const String & value);
template void NamedCollection::setOrUpdate<String, false>(const NamedCollection::Key & key, const String & value);
template void NamedCollection::setOrUpdate<UInt64, true>(const NamedCollection::Key & key, const UInt64 & value);
template void NamedCollection::setOrUpdate<UInt64, false>(const NamedCollection::Key & key, const UInt64 & value);
template void NamedCollection::setOrUpdate<Int64, true>(const NamedCollection::Key & key, const Int64 & value);
template void NamedCollection::setOrUpdate<Int64, false>(const NamedCollection::Key & key, const Int64 & value);
template void NamedCollection::setOrUpdate<Float64, true>(const NamedCollection::Key & key, const Float64 & value);
template void NamedCollection::setOrUpdate<Float64, false>(const NamedCollection::Key & key, const Float64 & value);
template void NamedCollection::remove<true>(const Key & key);
template void NamedCollection::remove<false>(const Key & key);
}

View File

@ -1,15 +1,13 @@
#pragma once
#include <Interpreters/Context.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Storages/NamedCollections_fwd.h>
#include <Storages/NamedCollectionUtils.h>
namespace Poco { namespace Util { class AbstractConfiguration; } }
namespace DB
{
class NamedCollection;
using NamedCollectionPtr = std::shared_ptr<const NamedCollection>;
/**
* Class to represent arbitrary-structured named collection object.
* It can be defined via config or via SQL command.
@ -22,40 +20,58 @@ using NamedCollectionPtr = std::shared_ptr<const NamedCollection>;
*/
class NamedCollection
{
private:
class Impl;
using ImplPtr = std::unique_ptr<Impl>;
ImplPtr pimpl;
public:
using Key = std::string;
using Keys = std::set<Key>;
using SourceId = NamedCollectionUtils::SourceId;
static NamedCollectionPtr create(
const Poco::Util::AbstractConfiguration & config,
const std::string & collection_name);
NamedCollection(
static MutableNamedCollectionPtr create(
const Poco::Util::AbstractConfiguration & config,
const std::string & collection_name,
const std::string & collection_path,
const Keys & keys);
explicit NamedCollection(ImplPtr pimpl_);
const Keys & keys,
SourceId source_id_,
bool is_mutable_);
template <typename T> T get(const Key & key) const;
template <typename T> T getOrDefault(const Key & key, const T & default_value) const;
template <typename T> void set(const Key & key, const T & value, bool update_if_exists = false);
std::unique_lock<std::mutex> lock();
void remove(const Key & key);
template <typename T, bool locked = false> void set(const Key & key, const T & value);
std::shared_ptr<NamedCollection> duplicate() const;
template <typename T, bool locked = false> void setOrUpdate(const Key & key, const T & value);
template <bool locked = false> void remove(const Key & key);
MutableNamedCollectionPtr duplicate() const;
Keys getKeys() const;
std::string dumpStructure() const;
bool isMutable() const { return is_mutable; }
SourceId getSourceId() const { return source_id; }
private:
class Impl;
using ImplPtr = std::unique_ptr<Impl>;
NamedCollection(
ImplPtr pimpl_,
const std::string & collection_name,
SourceId source_id,
bool is_mutable);
void assertMutable() const;
ImplPtr pimpl;
const std::string collection_name;
const SourceId source_id;
const bool is_mutable;
mutable std::mutex mutex;
};
/**
@ -66,42 +82,51 @@ class NamedCollectionFactory : boost::noncopyable
public:
static NamedCollectionFactory & instance();
void initialize(const Poco::Util::AbstractConfiguration & config_);
void reload(const Poco::Util::AbstractConfiguration & config_);
bool exists(const std::string & collection_name) const;
NamedCollectionPtr get(const std::string & collection_name) const;
NamedCollectionPtr tryGet(const std::string & collection_name) const;
void add(
const std::string & collection_name,
NamedCollectionPtr collection);
MutableNamedCollectionPtr getMutable(const std::string & collection_name) const;
void add(const std::string & collection_name, MutableNamedCollectionPtr collection);
void add(NamedCollectionsMap collections);
void update(NamedCollectionsMap collections);
void remove(const std::string & collection_name);
using NamedCollections = std::unordered_map<std::string, NamedCollectionPtr>;
NamedCollections getAll() const;
void removeIfExists(const std::string & collection_name);
void removeById(NamedCollectionUtils::SourceId id);
NamedCollectionsMap getAll() const;
private:
void assertInitialized(std::lock_guard<std::mutex> & lock) const;
NamedCollectionPtr getImpl(
const std::string & collection_name,
std::lock_guard<std::mutex> & lock) const;
bool existsUnlocked(
const std::string & collection_name,
std::lock_guard<std::mutex> & lock) const;
mutable NamedCollections loaded_named_collections;
MutableNamedCollectionPtr tryGetUnlocked(
const std::string & collection_name,
std::lock_guard<std::mutex> & lock) const;
const Poco::Util::AbstractConfiguration * config;
void addUnlocked(
const std::string & collection_name,
MutableNamedCollectionPtr collection,
std::lock_guard<std::mutex> & lock);
bool removeIfExistsUnlocked(
const std::string & collection_name,
std::lock_guard<std::mutex> & lock);
mutable NamedCollectionsMap loaded_named_collections;
bool is_initialized = false;
mutable std::mutex mutex;
bool is_initialized = false;
};
}

View File

@ -0,0 +1,12 @@
#pragma once
#include <map>
namespace DB
{
class NamedCollection;
using NamedCollectionPtr = std::shared_ptr<const NamedCollection>;
using MutableNamedCollectionPtr = std::shared_ptr<NamedCollection>;
using NamedCollectionsMap = std::map<std::string, MutableNamedCollectionPtr>;
}

View File

@ -1,5 +1,6 @@
#include <Common/tests/gtest_global_context.h>
#include <Storages/NamedCollections.h>
#include <Storages/NamedCollectionUtils.h>
#include <Poco/Util/XMLConfiguration.h>
#include <Poco/DOM/DOMParser.h>
#include <gtest/gtest.h>
@ -28,7 +29,7 @@ TEST(NamedCollections, SimpleConfig)
Poco::AutoPtr<Poco::XML::Document> document = dom_parser.parseString(xml);
Poco::AutoPtr<Poco::Util::XMLConfiguration> config = new Poco::Util::XMLConfiguration(document);
NamedCollectionFactory::instance().initialize(*config);
NamedCollectionUtils::loadFromConfig(*config);
ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection1"));
ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection2"));
@ -76,16 +77,16 @@ key5: 5
key6: 6.6
)CONFIG");
collection2_copy->set<String>("key4", "value44", true);
ASSERT_TRUE(collection2_copy->get<String>("key4") == "value44");
ASSERT_TRUE(collection2->get<String>("key4") == "value4");
collection2_copy->setOrUpdate<String>("key4", "value44");
ASSERT_EQ(collection2_copy->get<String>("key4"), "value44");
ASSERT_EQ(collection2->get<String>("key4"), "value4");
collection2_copy->remove("key4");
ASSERT_TRUE(collection2_copy->getOrDefault<String>("key4", "N") == "N");
ASSERT_TRUE(collection2->getOrDefault<String>("key4", "N") == "value4");
ASSERT_EQ(collection2_copy->getOrDefault<String>("key4", "N"), "N");
ASSERT_EQ(collection2->getOrDefault<String>("key4", "N"), "value4");
collection2_copy->set<String>("key4", "value45");
ASSERT_TRUE(collection2_copy->getOrDefault<String>("key4", "N") == "value45");
collection2_copy->setOrUpdate<String>("key4", "value45");
ASSERT_EQ(collection2_copy->getOrDefault<String>("key4", "N"), "value45");
NamedCollectionFactory::instance().remove("collection2_copy");
ASSERT_FALSE(NamedCollectionFactory::instance().exists("collection2_copy"));
@ -97,7 +98,7 @@ TEST(NamedCollections, NestedConfig)
{
std::string xml(R"CONFIG(<clickhouse>
<named_collections>
<collection1>
<collection3>
<key1>
<key1_1>value1</key1_1>
</key1>
@ -110,21 +111,22 @@ TEST(NamedCollections, NestedConfig)
</key2_3>
</key2_2>
</key2>
</collection1>
</collection3>
</named_collections>
</clickhouse>)CONFIG");
Poco::XML::DOMParser dom_parser;
Poco::AutoPtr<Poco::XML::Document> document = dom_parser.parseString(xml);
Poco::AutoPtr<Poco::Util::XMLConfiguration> config = new Poco::Util::XMLConfiguration(document);
NamedCollectionFactory::instance().reload(*config);
ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection1"));
NamedCollectionUtils::loadFromConfig(*config);
auto collection1 = NamedCollectionFactory::instance().get("collection1");
ASSERT_TRUE(collection1 != nullptr);
ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection3"));
ASSERT_EQ(collection1->dumpStructure(),
auto collection = NamedCollectionFactory::instance().get("collection3");
ASSERT_TRUE(collection != nullptr);
ASSERT_EQ(collection->dumpStructure(),
R"CONFIG(key1:
key1_1: value1
key2:
@ -135,9 +137,9 @@ key2:
key2_5: 5
)CONFIG");
ASSERT_EQ(collection1->get<String>("key1.key1_1"), "value1");
ASSERT_EQ(collection1->get<String>("key2.key2_1"), "value2_1");
ASSERT_EQ(collection1->get<Int64>("key2.key2_2.key2_3.key2_4"), 4);
ASSERT_EQ(collection1->get<Int64>("key2.key2_2.key2_3.key2_5"), 5);
ASSERT_EQ(collection->get<String>("key1.key1_1"), "value1");
ASSERT_EQ(collection->get<String>("key2.key2_1"), "value2_1");
ASSERT_EQ(collection->get<Int64>("key2.key2_2.key2_3.key2_4"), 4);
ASSERT_EQ(collection->get<Int64>("key2.key2_2.key2_3.key2_5"), 5);
}

View File

@ -13,14 +13,16 @@ import jwt
import requests # type: ignore
import boto3 # type: ignore
PULL_REQUEST_CI = "PullRequestCI"
NEED_RERUN_ON_EDITED = {
"PullRequestCI",
"DocsCheck",
}
NEED_RERUN_OR_CANCELL_WORKFLOWS = {
PULL_REQUEST_CI,
"DocsCheck",
"DocsReleaseChecks",
"BackportPR",
}
}.union(NEED_RERUN_ON_EDITED)
MAX_RETRY = 5
@ -334,7 +336,7 @@ def main(event):
most_recent_workflow = workflow_descriptions[-1]
if (
most_recent_workflow.status == "completed"
and most_recent_workflow.name == PULL_REQUEST_CI
and most_recent_workflow.name in NEED_RERUN_ON_EDITED
):
print(
"The PR's body is changed and workflow is finished. "

View File

@ -0,0 +1,7 @@
<clickhouse>
<named_collections>
<collection1>
<key1>value1</key1>
</collection1>
</named_collections>
</clickhouse>

View File

@ -0,0 +1,13 @@
<clickhouse>
<users>
<default>
<password></password>
<networks>
<ip>::/0</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
<show_named_collections>1</show_named_collections>
</default>
</users>
</clickhouse>

View File

@ -0,0 +1,200 @@
import logging
import pytest
import os
import time
from helpers.cluster import ClickHouseCluster
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
NAMED_COLLECTIONS_CONFIG = os.path.join(
SCRIPT_DIR, "./configs/config.d/named_collections.xml"
)
@pytest.fixture(scope="module")
def cluster():
try:
cluster = ClickHouseCluster(__file__)
cluster.add_instance(
"node",
main_configs=[
"configs/config.d/named_collections.xml",
],
user_configs=[
"configs/users.d/users.xml",
],
stay_alive=True,
)
logging.info("Starting cluster...")
cluster.start()
logging.info("Cluster started")
yield cluster
finally:
cluster.shutdown()
def replace_config(node, old, new):
node.replace_in_config(
"/etc/clickhouse-server/config.d/named_collections.xml",
old,
new,
)
def test_config_reload(cluster):
node = cluster.instances["node"]
assert (
"collection1" == node.query("select name from system.named_collections").strip()
)
assert (
"['key1']"
== node.query(
"select mapKeys(collection) from system.named_collections where name = 'collection1'"
).strip()
)
assert (
"value1"
== node.query(
"select collection['key1'] from system.named_collections where name = 'collection1'"
).strip()
)
replace_config(node, "value1", "value2")
node.query("SYSTEM RELOAD CONFIG")
assert (
"['key1']"
== node.query(
"select mapKeys(collection) from system.named_collections where name = 'collection1'"
).strip()
)
assert (
"value2"
== node.query(
"select collection['key1'] from system.named_collections where name = 'collection1'"
).strip()
)
def test_sql_commands(cluster):
node = cluster.instances["node"]
assert "1" == node.query("select count() from system.named_collections").strip()
node.query("CREATE NAMED COLLECTION collection2 AS key1=1, key2='value2'")
def check_created():
assert (
"collection1\ncollection2"
== node.query("select name from system.named_collections").strip()
)
assert (
"['key1','key2']"
== node.query(
"select mapKeys(collection) from system.named_collections where name = 'collection2'"
).strip()
)
assert (
"1"
== node.query(
"select collection['key1'] from system.named_collections where name = 'collection2'"
).strip()
)
assert (
"value2"
== node.query(
"select collection['key2'] from system.named_collections where name = 'collection2'"
).strip()
)
check_created()
node.restart_clickhouse()
check_created()
node.query("ALTER NAMED COLLECTION collection2 SET key1=4, key3='value3'")
def check_altered():
assert (
"['key1','key2','key3']"
== node.query(
"select mapKeys(collection) from system.named_collections where name = 'collection2'"
).strip()
)
assert (
"4"
== node.query(
"select collection['key1'] from system.named_collections where name = 'collection2'"
).strip()
)
assert (
"value3"
== node.query(
"select collection['key3'] from system.named_collections where name = 'collection2'"
).strip()
)
check_altered()
node.restart_clickhouse()
check_altered()
node.query("ALTER NAMED COLLECTION collection2 DELETE key2")
def check_deleted():
assert (
"['key1','key3']"
== node.query(
"select mapKeys(collection) from system.named_collections where name = 'collection2'"
).strip()
)
check_deleted()
node.restart_clickhouse()
check_deleted()
node.query(
"ALTER NAMED COLLECTION collection2 SET key3=3, key4='value4' DELETE key1"
)
def check_altered_and_deleted():
assert (
"['key3','key4']"
== node.query(
"select mapKeys(collection) from system.named_collections where name = 'collection2'"
).strip()
)
assert (
"3"
== node.query(
"select collection['key3'] from system.named_collections where name = 'collection2'"
).strip()
)
assert (
"value4"
== node.query(
"select collection['key4'] from system.named_collections where name = 'collection2'"
).strip()
)
check_altered_and_deleted()
node.restart_clickhouse()
check_altered_and_deleted()
node.query("DROP NAMED COLLECTION collection2")
def check_dropped():
assert "1" == node.query("select count() from system.named_collections").strip()
assert (
"collection1"
== node.query("select name from system.named_collections").strip()
)
check_dropped()
node.restart_clickhouse()
check_dropped()

View File

@ -39,6 +39,7 @@ ALTER MOVE PARTITION ['ALTER MOVE PART','MOVE PARTITION','MOVE PART'] TABLE ALTE
ALTER FETCH PARTITION ['ALTER FETCH PART','FETCH PARTITION'] TABLE ALTER TABLE
ALTER FREEZE PARTITION ['FREEZE PARTITION','UNFREEZE'] TABLE ALTER TABLE
ALTER DATABASE SETTINGS ['ALTER DATABASE SETTING','ALTER MODIFY DATABASE SETTING','MODIFY DATABASE SETTING'] DATABASE ALTER DATABASE
ALTER NAMED COLLECTION [] \N ALTER
ALTER TABLE [] \N ALTER
ALTER DATABASE [] \N ALTER
ALTER VIEW REFRESH ['ALTER LIVE VIEW REFRESH','REFRESH VIEW'] VIEW ALTER VIEW
@ -51,12 +52,14 @@ CREATE VIEW [] VIEW CREATE
CREATE DICTIONARY [] DICTIONARY CREATE
CREATE TEMPORARY TABLE [] GLOBAL CREATE
CREATE FUNCTION [] GLOBAL CREATE
CREATE NAMED COLLECTION [] GLOBAL CREATE
CREATE [] \N ALL
DROP DATABASE [] DATABASE DROP
DROP TABLE [] TABLE DROP
DROP VIEW [] VIEW DROP
DROP DICTIONARY [] DICTIONARY DROP
DROP FUNCTION [] GLOBAL DROP
DROP NAMED COLLECTION [] GLOBAL DROP
DROP [] \N ALL
TRUNCATE ['TRUNCATE TABLE'] TABLE ALL
OPTIMIZE ['OPTIMIZE TABLE'] TABLE ALL

View File

@ -0,0 +1,21 @@
# clickhouse-client
concatAssumeInjective: OK
ReplacingMergeTree: OK
JSONEachRow: OK
clusterAllReplicas: OK
SimpleAggregateFunction: OK
write_ahead_log_interval_ms_to_fsync: OK
max_concurrent_queries_for_all_users: OK
test_shard_localhost: OK
default_path_test: OK
default: OK
uniqCombined64ForEach: OK
system: OK
aggregate_function_combinators: OK
primary_key_bytes_in_memory_allocated: OK
# clickhouse-local
concatAssumeInjective: OK
ReplacingMergeTree: OK
JSONEachRow: OK
clusterAllReplicas: OK
SimpleAggregateFunction: OK

View File

@ -5,9 +5,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
SCRIPT_PATH="$CURDIR/$CLICKHOUSE_TEST_UNIQUE_NAME.generated-expect"
# NOTE: database = $CLICKHOUSE_DATABASE is superfluous
function test_completion_word_client()
function test_completion_word()
{
local w=$1 && shift
@ -15,10 +17,20 @@ function test_completion_word_client()
local compword_begin=${w:0:$((w_len-3))}
local compword_end=${w:$((w_len-3))}
# NOTE: here and below you should escape variables of the expect.
timeout 60s expect << EOF
# NOTE:
# - here and below you should escape variables of the expect.
# - you should not use "expect <<..." since in this case timeout/eof will
# not work (I guess due to attached stdin)
cat > "$SCRIPT_PATH" << EOF
# NOTE: log will be appended
exp_internal -f $CLICKHOUSE_TMP/$(basename "${BASH_SOURCE[0]}").debuglog 0
# NOTE: when expect have EOF on stdin it also closes stdout, so let's reopen it
# again for logging
set stdout_channel [open "/dev/stdout" w]
log_user 0
set timeout 3
set timeout 60
match_max 100000
expect_after {
# Do not ignore eof from expect
@ -27,7 +39,7 @@ expect_after {
timeout { exit 1 }
}
spawn bash -c "$CLICKHOUSE_CLIENT_BINARY $CLICKHOUSE_CLIENT_OPT"
spawn bash -c "$*"
expect ":) "
# Make a query
@ -36,10 +48,12 @@ expect "SET $compword_begin"
# Wait for suggestions to load, they are loaded in background
set is_done 0
set timeout 1
while {\$is_done == 0} {
send -- "\\t"
expect {
"$compword_begin$compword_end" {
puts \$stdout_channel "$compword_begin$compword_end: OK"
set is_done 1
}
default {
@ -48,9 +62,18 @@ while {\$is_done == 0} {
}
}
close \$stdout_channel
send -- "\\3\\4"
expect eof
EOF
# NOTE: run expect under timeout since there is while loop that is not
# limited with timeout.
#
# NOTE: cat is required to serialize stdout for expect (without this pipe
# it will reopen the file again, and the output will be mixed).
timeout 2m expect -f "$SCRIPT_PATH" | cat
}
# last 3 bytes will be completed,
@ -90,53 +113,6 @@ client_compwords_positive=(
# FIXME: none
)
function test_completion_word_local()
{
local w=$1 && shift
local w_len=${#w}
local compword_begin=${w:0:$((w_len-3))}
local compword_end=${w:$((w_len-3))}
# NOTE: here and below you should escape variables of the expect.
timeout 60s expect << EOF
log_user 0
set timeout 3
match_max 100000
expect_after {
# Do not ignore eof from expect
eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
timeout { exit 1 }
}
spawn bash -c "$CLICKHOUSE_LOCAL"
expect ":) "
# Make a query
send -- "SET $compword_begin"
expect "SET $compword_begin"
# Wait for suggestions to load, they are loaded in background
set is_done 0
while {\$is_done == 0} {
send -- "\\t"
expect {
"$compword_begin$compword_end" {
set is_done 1
}
default {
sleep 1
}
}
}
send -- "\\3\\4"
expect eof
EOF
}
local_compwords_positive=(
# system.functions
concatAssumeInjective
@ -150,12 +126,15 @@ local_compwords_positive=(
SimpleAggregateFunction
)
echo "# clickhouse-client"
for w in "${client_compwords_positive[@]}"; do
test_completion_word_client "$w" || echo "[FAIL] $w (positive)"
test_completion_word "$w" "$CLICKHOUSE_CLIENT"
done
echo "# clickhouse-local"
for w in "${local_compwords_positive[@]}"; do
test_completion_word "$w" "$CLICKHOUSE_LOCAL"
done
for w in "${local_compwords_positive[@]}"; do
test_completion_word_local "$w" || echo "[FAIL] $w (positive)"
done
rm -f "${SCRIPT_PATH:?}"
exit 0

View File

@ -284,7 +284,7 @@ CREATE TABLE system.grants
(
`user_name` Nullable(String),
`role_name` Nullable(String),
`access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER TABLE' = 41, 'ALTER DATABASE' = 42, 'ALTER VIEW REFRESH' = 43, 'ALTER VIEW MODIFY QUERY' = 44, 'ALTER VIEW' = 45, 'ALTER' = 46, 'CREATE DATABASE' = 47, 'CREATE TABLE' = 48, 'CREATE VIEW' = 49, 'CREATE DICTIONARY' = 50, 'CREATE TEMPORARY TABLE' = 51, 'CREATE FUNCTION' = 52, 'CREATE' = 53, 'DROP DATABASE' = 54, 'DROP TABLE' = 55, 'DROP VIEW' = 56, 'DROP DICTIONARY' = 57, 'DROP FUNCTION' = 58, 'DROP' = 59, 'TRUNCATE' = 60, 'OPTIMIZE' = 61, 'BACKUP' = 62, 'KILL QUERY' = 63, 'KILL TRANSACTION' = 64, 'MOVE PARTITION BETWEEN SHARDS' = 65, 'CREATE USER' = 66, 'ALTER USER' = 67, 'DROP USER' = 68, 'CREATE ROLE' = 69, 'ALTER ROLE' = 70, 'DROP ROLE' = 71, 'ROLE ADMIN' = 72, 'CREATE ROW POLICY' = 73, 'ALTER ROW POLICY' = 74, 'DROP ROW POLICY' = 75, 'CREATE QUOTA' = 76, 'ALTER QUOTA' = 77, 'DROP QUOTA' = 78, 'CREATE SETTINGS PROFILE' = 79, 'ALTER SETTINGS PROFILE' = 80, 'DROP SETTINGS PROFILE' = 81, 'SHOW USERS' = 82, 'SHOW ROLES' = 83, 'SHOW ROW POLICIES' = 84, 'SHOW QUOTAS' = 85, 'SHOW SETTINGS PROFILES' = 86, 'SHOW ACCESS' = 87, 'SHOW NAMED COLLECTIONS' = 88, 'ACCESS MANAGEMENT' = 89, 'SYSTEM SHUTDOWN' = 90, 'SYSTEM DROP DNS CACHE' = 91, 'SYSTEM DROP MARK CACHE' = 92, 'SYSTEM DROP UNCOMPRESSED CACHE' = 93, 'SYSTEM DROP MMAP CACHE' = 94, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 95, 'SYSTEM DROP FILESYSTEM CACHE' = 96, 'SYSTEM DROP SCHEMA CACHE' = 97, 'SYSTEM DROP CACHE' = 98, 'SYSTEM RELOAD CONFIG' = 99, 'SYSTEM RELOAD USERS' = 100, 'SYSTEM RELOAD SYMBOLS' = 101, 'SYSTEM RELOAD DICTIONARY' = 102, 'SYSTEM RELOAD MODEL' = 103, 'SYSTEM RELOAD FUNCTION' = 104, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 105, 'SYSTEM RELOAD' = 106, 'SYSTEM RESTART DISK' = 107, 'SYSTEM MERGES' = 108, 'SYSTEM TTL MERGES' = 109, 'SYSTEM FETCHES' = 110, 'SYSTEM MOVES' = 111, 'SYSTEM DISTRIBUTED SENDS' = 112, 'SYSTEM REPLICATED SENDS' = 113, 'SYSTEM SENDS' = 114, 'SYSTEM REPLICATION QUEUES' = 115, 'SYSTEM DROP REPLICA' = 116, 'SYSTEM SYNC REPLICA' = 117, 'SYSTEM RESTART REPLICA' = 118, 'SYSTEM RESTORE REPLICA' = 119, 'SYSTEM SYNC DATABASE REPLICA' = 120, 'SYSTEM SYNC TRANSACTION LOG' = 121, 'SYSTEM FLUSH DISTRIBUTED' = 122, 'SYSTEM FLUSH LOGS' = 123, 'SYSTEM FLUSH' = 124, 'SYSTEM THREAD FUZZER' = 125, 'SYSTEM UNFREEZE' = 126, 'SYSTEM' = 127, 'dictGet' = 128, 'addressToLine' = 129, 'addressToLineWithInlines' = 130, 'addressToSymbol' = 131, 'demangle' = 132, 'INTROSPECTION' = 133, 'FILE' = 134, 'URL' = 135, 'REMOTE' = 136, 'MONGO' = 137, 'MEILISEARCH' = 138, 'MYSQL' = 139, 'POSTGRES' = 140, 'SQLITE' = 141, 'ODBC' = 142, 'JDBC' = 143, 'HDFS' = 144, 'S3' = 145, 'HIVE' = 146, 'SOURCES' = 147, 'CLUSTER' = 148, 'ALL' = 149, 'NONE' = 150),
`access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 98, 'SYSTEM DROP FILESYSTEM CACHE' = 99, 'SYSTEM DROP SCHEMA CACHE' = 100, 'SYSTEM DROP CACHE' = 101, 'SYSTEM RELOAD CONFIG' = 102, 'SYSTEM RELOAD USERS' = 103, 'SYSTEM RELOAD SYMBOLS' = 104, 'SYSTEM RELOAD DICTIONARY' = 105, 'SYSTEM RELOAD MODEL' = 106, 'SYSTEM RELOAD FUNCTION' = 107, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 108, 'SYSTEM RELOAD' = 109, 'SYSTEM RESTART DISK' = 110, 'SYSTEM MERGES' = 111, 'SYSTEM TTL MERGES' = 112, 'SYSTEM FETCHES' = 113, 'SYSTEM MOVES' = 114, 'SYSTEM DISTRIBUTED SENDS' = 115, 'SYSTEM REPLICATED SENDS' = 116, 'SYSTEM SENDS' = 117, 'SYSTEM REPLICATION QUEUES' = 118, 'SYSTEM DROP REPLICA' = 119, 'SYSTEM SYNC REPLICA' = 120, 'SYSTEM RESTART REPLICA' = 121, 'SYSTEM RESTORE REPLICA' = 122, 'SYSTEM SYNC DATABASE REPLICA' = 123, 'SYSTEM SYNC TRANSACTION LOG' = 124, 'SYSTEM FLUSH DISTRIBUTED' = 125, 'SYSTEM FLUSH LOGS' = 126, 'SYSTEM FLUSH' = 127, 'SYSTEM THREAD FUZZER' = 128, 'SYSTEM UNFREEZE' = 129, 'SYSTEM' = 130, 'dictGet' = 131, 'addressToLine' = 132, 'addressToLineWithInlines' = 133, 'addressToSymbol' = 134, 'demangle' = 135, 'INTROSPECTION' = 136, 'FILE' = 137, 'URL' = 138, 'REMOTE' = 139, 'MONGO' = 140, 'MEILISEARCH' = 141, 'MYSQL' = 142, 'POSTGRES' = 143, 'SQLITE' = 144, 'ODBC' = 145, 'JDBC' = 146, 'HDFS' = 147, 'S3' = 148, 'HIVE' = 149, 'SOURCES' = 150, 'CLUSTER' = 151, 'ALL' = 152, 'NONE' = 153),
`database` Nullable(String),
`table` Nullable(String),
`column` Nullable(String),
@ -560,10 +560,10 @@ ENGINE = SystemPartsColumns
COMMENT 'SYSTEM TABLE is built on the fly.'
CREATE TABLE system.privileges
(
`privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER TABLE' = 41, 'ALTER DATABASE' = 42, 'ALTER VIEW REFRESH' = 43, 'ALTER VIEW MODIFY QUERY' = 44, 'ALTER VIEW' = 45, 'ALTER' = 46, 'CREATE DATABASE' = 47, 'CREATE TABLE' = 48, 'CREATE VIEW' = 49, 'CREATE DICTIONARY' = 50, 'CREATE TEMPORARY TABLE' = 51, 'CREATE FUNCTION' = 52, 'CREATE' = 53, 'DROP DATABASE' = 54, 'DROP TABLE' = 55, 'DROP VIEW' = 56, 'DROP DICTIONARY' = 57, 'DROP FUNCTION' = 58, 'DROP' = 59, 'TRUNCATE' = 60, 'OPTIMIZE' = 61, 'BACKUP' = 62, 'KILL QUERY' = 63, 'KILL TRANSACTION' = 64, 'MOVE PARTITION BETWEEN SHARDS' = 65, 'CREATE USER' = 66, 'ALTER USER' = 67, 'DROP USER' = 68, 'CREATE ROLE' = 69, 'ALTER ROLE' = 70, 'DROP ROLE' = 71, 'ROLE ADMIN' = 72, 'CREATE ROW POLICY' = 73, 'ALTER ROW POLICY' = 74, 'DROP ROW POLICY' = 75, 'CREATE QUOTA' = 76, 'ALTER QUOTA' = 77, 'DROP QUOTA' = 78, 'CREATE SETTINGS PROFILE' = 79, 'ALTER SETTINGS PROFILE' = 80, 'DROP SETTINGS PROFILE' = 81, 'SHOW USERS' = 82, 'SHOW ROLES' = 83, 'SHOW ROW POLICIES' = 84, 'SHOW QUOTAS' = 85, 'SHOW SETTINGS PROFILES' = 86, 'SHOW ACCESS' = 87, 'SHOW NAMED COLLECTIONS' = 88, 'ACCESS MANAGEMENT' = 89, 'SYSTEM SHUTDOWN' = 90, 'SYSTEM DROP DNS CACHE' = 91, 'SYSTEM DROP MARK CACHE' = 92, 'SYSTEM DROP UNCOMPRESSED CACHE' = 93, 'SYSTEM DROP MMAP CACHE' = 94, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 95, 'SYSTEM DROP FILESYSTEM CACHE' = 96, 'SYSTEM DROP SCHEMA CACHE' = 97, 'SYSTEM DROP CACHE' = 98, 'SYSTEM RELOAD CONFIG' = 99, 'SYSTEM RELOAD USERS' = 100, 'SYSTEM RELOAD SYMBOLS' = 101, 'SYSTEM RELOAD DICTIONARY' = 102, 'SYSTEM RELOAD MODEL' = 103, 'SYSTEM RELOAD FUNCTION' = 104, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 105, 'SYSTEM RELOAD' = 106, 'SYSTEM RESTART DISK' = 107, 'SYSTEM MERGES' = 108, 'SYSTEM TTL MERGES' = 109, 'SYSTEM FETCHES' = 110, 'SYSTEM MOVES' = 111, 'SYSTEM DISTRIBUTED SENDS' = 112, 'SYSTEM REPLICATED SENDS' = 113, 'SYSTEM SENDS' = 114, 'SYSTEM REPLICATION QUEUES' = 115, 'SYSTEM DROP REPLICA' = 116, 'SYSTEM SYNC REPLICA' = 117, 'SYSTEM RESTART REPLICA' = 118, 'SYSTEM RESTORE REPLICA' = 119, 'SYSTEM SYNC DATABASE REPLICA' = 120, 'SYSTEM SYNC TRANSACTION LOG' = 121, 'SYSTEM FLUSH DISTRIBUTED' = 122, 'SYSTEM FLUSH LOGS' = 123, 'SYSTEM FLUSH' = 124, 'SYSTEM THREAD FUZZER' = 125, 'SYSTEM UNFREEZE' = 126, 'SYSTEM' = 127, 'dictGet' = 128, 'addressToLine' = 129, 'addressToLineWithInlines' = 130, 'addressToSymbol' = 131, 'demangle' = 132, 'INTROSPECTION' = 133, 'FILE' = 134, 'URL' = 135, 'REMOTE' = 136, 'MONGO' = 137, 'MEILISEARCH' = 138, 'MYSQL' = 139, 'POSTGRES' = 140, 'SQLITE' = 141, 'ODBC' = 142, 'JDBC' = 143, 'HDFS' = 144, 'S3' = 145, 'HIVE' = 146, 'SOURCES' = 147, 'CLUSTER' = 148, 'ALL' = 149, 'NONE' = 150),
`privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 98, 'SYSTEM DROP FILESYSTEM CACHE' = 99, 'SYSTEM DROP SCHEMA CACHE' = 100, 'SYSTEM DROP CACHE' = 101, 'SYSTEM RELOAD CONFIG' = 102, 'SYSTEM RELOAD USERS' = 103, 'SYSTEM RELOAD SYMBOLS' = 104, 'SYSTEM RELOAD DICTIONARY' = 105, 'SYSTEM RELOAD MODEL' = 106, 'SYSTEM RELOAD FUNCTION' = 107, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 108, 'SYSTEM RELOAD' = 109, 'SYSTEM RESTART DISK' = 110, 'SYSTEM MERGES' = 111, 'SYSTEM TTL MERGES' = 112, 'SYSTEM FETCHES' = 113, 'SYSTEM MOVES' = 114, 'SYSTEM DISTRIBUTED SENDS' = 115, 'SYSTEM REPLICATED SENDS' = 116, 'SYSTEM SENDS' = 117, 'SYSTEM REPLICATION QUEUES' = 118, 'SYSTEM DROP REPLICA' = 119, 'SYSTEM SYNC REPLICA' = 120, 'SYSTEM RESTART REPLICA' = 121, 'SYSTEM RESTORE REPLICA' = 122, 'SYSTEM SYNC DATABASE REPLICA' = 123, 'SYSTEM SYNC TRANSACTION LOG' = 124, 'SYSTEM FLUSH DISTRIBUTED' = 125, 'SYSTEM FLUSH LOGS' = 126, 'SYSTEM FLUSH' = 127, 'SYSTEM THREAD FUZZER' = 128, 'SYSTEM UNFREEZE' = 129, 'SYSTEM' = 130, 'dictGet' = 131, 'addressToLine' = 132, 'addressToLineWithInlines' = 133, 'addressToSymbol' = 134, 'demangle' = 135, 'INTROSPECTION' = 136, 'FILE' = 137, 'URL' = 138, 'REMOTE' = 139, 'MONGO' = 140, 'MEILISEARCH' = 141, 'MYSQL' = 142, 'POSTGRES' = 143, 'SQLITE' = 144, 'ODBC' = 145, 'JDBC' = 146, 'HDFS' = 147, 'S3' = 148, 'HIVE' = 149, 'SOURCES' = 150, 'CLUSTER' = 151, 'ALL' = 152, 'NONE' = 153),
`aliases` Array(String),
`level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5)),
`parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER TABLE' = 41, 'ALTER DATABASE' = 42, 'ALTER VIEW REFRESH' = 43, 'ALTER VIEW MODIFY QUERY' = 44, 'ALTER VIEW' = 45, 'ALTER' = 46, 'CREATE DATABASE' = 47, 'CREATE TABLE' = 48, 'CREATE VIEW' = 49, 'CREATE DICTIONARY' = 50, 'CREATE TEMPORARY TABLE' = 51, 'CREATE FUNCTION' = 52, 'CREATE' = 53, 'DROP DATABASE' = 54, 'DROP TABLE' = 55, 'DROP VIEW' = 56, 'DROP DICTIONARY' = 57, 'DROP FUNCTION' = 58, 'DROP' = 59, 'TRUNCATE' = 60, 'OPTIMIZE' = 61, 'BACKUP' = 62, 'KILL QUERY' = 63, 'KILL TRANSACTION' = 64, 'MOVE PARTITION BETWEEN SHARDS' = 65, 'CREATE USER' = 66, 'ALTER USER' = 67, 'DROP USER' = 68, 'CREATE ROLE' = 69, 'ALTER ROLE' = 70, 'DROP ROLE' = 71, 'ROLE ADMIN' = 72, 'CREATE ROW POLICY' = 73, 'ALTER ROW POLICY' = 74, 'DROP ROW POLICY' = 75, 'CREATE QUOTA' = 76, 'ALTER QUOTA' = 77, 'DROP QUOTA' = 78, 'CREATE SETTINGS PROFILE' = 79, 'ALTER SETTINGS PROFILE' = 80, 'DROP SETTINGS PROFILE' = 81, 'SHOW USERS' = 82, 'SHOW ROLES' = 83, 'SHOW ROW POLICIES' = 84, 'SHOW QUOTAS' = 85, 'SHOW SETTINGS PROFILES' = 86, 'SHOW ACCESS' = 87, 'SHOW NAMED COLLECTIONS' = 88, 'ACCESS MANAGEMENT' = 89, 'SYSTEM SHUTDOWN' = 90, 'SYSTEM DROP DNS CACHE' = 91, 'SYSTEM DROP MARK CACHE' = 92, 'SYSTEM DROP UNCOMPRESSED CACHE' = 93, 'SYSTEM DROP MMAP CACHE' = 94, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 95, 'SYSTEM DROP FILESYSTEM CACHE' = 96, 'SYSTEM DROP SCHEMA CACHE' = 97, 'SYSTEM DROP CACHE' = 98, 'SYSTEM RELOAD CONFIG' = 99, 'SYSTEM RELOAD USERS' = 100, 'SYSTEM RELOAD SYMBOLS' = 101, 'SYSTEM RELOAD DICTIONARY' = 102, 'SYSTEM RELOAD MODEL' = 103, 'SYSTEM RELOAD FUNCTION' = 104, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 105, 'SYSTEM RELOAD' = 106, 'SYSTEM RESTART DISK' = 107, 'SYSTEM MERGES' = 108, 'SYSTEM TTL MERGES' = 109, 'SYSTEM FETCHES' = 110, 'SYSTEM MOVES' = 111, 'SYSTEM DISTRIBUTED SENDS' = 112, 'SYSTEM REPLICATED SENDS' = 113, 'SYSTEM SENDS' = 114, 'SYSTEM REPLICATION QUEUES' = 115, 'SYSTEM DROP REPLICA' = 116, 'SYSTEM SYNC REPLICA' = 117, 'SYSTEM RESTART REPLICA' = 118, 'SYSTEM RESTORE REPLICA' = 119, 'SYSTEM SYNC DATABASE REPLICA' = 120, 'SYSTEM SYNC TRANSACTION LOG' = 121, 'SYSTEM FLUSH DISTRIBUTED' = 122, 'SYSTEM FLUSH LOGS' = 123, 'SYSTEM FLUSH' = 124, 'SYSTEM THREAD FUZZER' = 125, 'SYSTEM UNFREEZE' = 126, 'SYSTEM' = 127, 'dictGet' = 128, 'addressToLine' = 129, 'addressToLineWithInlines' = 130, 'addressToSymbol' = 131, 'demangle' = 132, 'INTROSPECTION' = 133, 'FILE' = 134, 'URL' = 135, 'REMOTE' = 136, 'MONGO' = 137, 'MEILISEARCH' = 138, 'MYSQL' = 139, 'POSTGRES' = 140, 'SQLITE' = 141, 'ODBC' = 142, 'JDBC' = 143, 'HDFS' = 144, 'S3' = 145, 'HIVE' = 146, 'SOURCES' = 147, 'CLUSTER' = 148, 'ALL' = 149, 'NONE' = 150))
`parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 98, 'SYSTEM DROP FILESYSTEM CACHE' = 99, 'SYSTEM DROP SCHEMA CACHE' = 100, 'SYSTEM DROP CACHE' = 101, 'SYSTEM RELOAD CONFIG' = 102, 'SYSTEM RELOAD USERS' = 103, 'SYSTEM RELOAD SYMBOLS' = 104, 'SYSTEM RELOAD DICTIONARY' = 105, 'SYSTEM RELOAD MODEL' = 106, 'SYSTEM RELOAD FUNCTION' = 107, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 108, 'SYSTEM RELOAD' = 109, 'SYSTEM RESTART DISK' = 110, 'SYSTEM MERGES' = 111, 'SYSTEM TTL MERGES' = 112, 'SYSTEM FETCHES' = 113, 'SYSTEM MOVES' = 114, 'SYSTEM DISTRIBUTED SENDS' = 115, 'SYSTEM REPLICATED SENDS' = 116, 'SYSTEM SENDS' = 117, 'SYSTEM REPLICATION QUEUES' = 118, 'SYSTEM DROP REPLICA' = 119, 'SYSTEM SYNC REPLICA' = 120, 'SYSTEM RESTART REPLICA' = 121, 'SYSTEM RESTORE REPLICA' = 122, 'SYSTEM SYNC DATABASE REPLICA' = 123, 'SYSTEM SYNC TRANSACTION LOG' = 124, 'SYSTEM FLUSH DISTRIBUTED' = 125, 'SYSTEM FLUSH LOGS' = 126, 'SYSTEM FLUSH' = 127, 'SYSTEM THREAD FUZZER' = 128, 'SYSTEM UNFREEZE' = 129, 'SYSTEM' = 130, 'dictGet' = 131, 'addressToLine' = 132, 'addressToLineWithInlines' = 133, 'addressToSymbol' = 134, 'demangle' = 135, 'INTROSPECTION' = 136, 'FILE' = 137, 'URL' = 138, 'REMOTE' = 139, 'MONGO' = 140, 'MEILISEARCH' = 141, 'MYSQL' = 142, 'POSTGRES' = 143, 'SQLITE' = 144, 'ODBC' = 145, 'JDBC' = 146, 'HDFS' = 147, 'S3' = 148, 'HIVE' = 149, 'SOURCES' = 150, 'CLUSTER' = 151, 'ALL' = 152, 'NONE' = 153))
)
ENGINE = SystemPrivileges
COMMENT 'SYSTEM TABLE is built on the fly.'

View File

@ -0,0 +1,2 @@
1
1 1

View File

@ -0,0 +1,18 @@
#!/usr/bin/env bash
# Tags: no-tsan, no-parallel
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
query_id="$RANDOM-$CLICKHOUSE_DATABASE"
${CLICKHOUSE_CLIENT} --query_id $query_id --query "SELECT 1 FORMAT Null SETTINGS trace_profile_events = 0"
${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS"
${CLICKHOUSE_CLIENT} --query "SELECT count() = 0 FROM system.trace_log WHERE query_id = '$query_id' AND trace_type = 'ProfileEvent'"
query_id="$RANDOM-$CLICKHOUSE_DATABASE"
${CLICKHOUSE_CLIENT} --query_id $query_id --query "SELECT 1 FORMAT Null SETTINGS trace_profile_events = 1"
${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS"
${CLICKHOUSE_CLIENT} --query "SELECT count() > 0, sum(empty(trace)) = 0 FROM system.trace_log WHERE query_id = '$query_id' AND trace_type = 'ProfileEvent'"