Merge branch 'master' into 100_percent_lld_11_for_clang_11

This commit is contained in:
alesapin 2020-09-29 11:14:47 +03:00
commit f104da48d2
54 changed files with 510 additions and 166 deletions

View File

@ -38,10 +38,10 @@ bool hasInputData()
}
LineReader::Suggest::WordsRange LineReader::Suggest::getCompletions(const String & prefix, size_t prefix_length) const
std::optional<LineReader::Suggest::WordsRange> LineReader::Suggest::getCompletions(const String & prefix, size_t prefix_length) const
{
if (!ready)
return std::make_pair(words.end(), words.end());
return std::nullopt;
std::string_view last_word;

View File

@ -4,6 +4,7 @@
#include <atomic>
#include <vector>
#include <optional>
class LineReader
{
@ -18,7 +19,7 @@ public:
std::atomic<bool> ready{false};
/// Get iterators for the matched range of words if any.
WordsRange getCompletions(const String & prefix, size_t prefix_length) const;
std::optional<WordsRange> getCompletions(const String & prefix, size_t prefix_length) const;
};
using Patterns = std::vector<const char *>;

View File

@ -30,7 +30,8 @@ static LineReader::Suggest::Words::const_iterator end;
static void findRange(const char * prefix, size_t prefix_length)
{
std::string prefix_str(prefix);
std::tie(pos, end) = suggest->getCompletions(prefix_str, prefix_length);
if (auto completions = suggest->getCompletions(prefix_str, prefix_length))
std::tie(pos, end) = *completions;
}
/// Iterates through matched range.

View File

@ -70,8 +70,9 @@ ReplxxLineReader::ReplxxLineReader(
auto callback = [&suggest] (const String & context, size_t context_size)
{
auto range = suggest.getCompletions(context, context_size);
return Replxx::completions_t(range.first, range.second);
if (auto range = suggest.getCompletions(context, context_size))
return Replxx::completions_t(range->first, range->second);
return Replxx::completions_t();
};
rx.set_completion_callback(callback);

View File

@ -23,6 +23,7 @@ ExtendedLogMessage ExtendedLogMessage::getFrom(const Poco::Message & base)
msg_ext.time_seconds = static_cast<UInt32>(tv.tv_sec);
msg_ext.time_microseconds = static_cast<UInt32>(tv.tv_usec);
msg_ext.time_in_microseconds = static_cast<UInt64>((tv.tv_sec) * 1000000U + (tv.tv_usec));
if (current_thread)
{

View File

@ -23,6 +23,7 @@ public:
uint32_t time_seconds = 0;
uint32_t time_microseconds = 0;
uint64_t time_in_microseconds = 0;
uint64_t thread_id = 0;
std::string query_id;

View File

@ -76,6 +76,7 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg)
TextLogElement elem;
elem.event_time = msg_ext.time_seconds;
elem.event_time_microseconds = msg_ext.time_in_microseconds;
elem.microseconds = msg_ext.time_microseconds;
elem.thread_name = getThreadName();

View File

@ -48,10 +48,15 @@ RUN apt-get update \
tzdata \
--yes --no-install-recommends
# Sanitizer options
# Sanitizer options for services (clickhouse-server)
RUN echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/environment; \
echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \
echo "MSAN_OPTIONS='abort_on_error=1'" >> /etc/environment; \
ln -s /usr/lib/llvm-${LLVM_VERSION}/bin/llvm-symbolizer /usr/bin/llvm-symbolizer;
# Sanitizer options for current shell (not current, but the one that will be spawned on "docker run")
# (but w/o verbosity for TSAN, otherwise test.reference will not match)
ENV TSAN_OPTIONS='halt_on_error=1 history_size=7'
ENV UBSAN_OPTIONS='print_stacktrace=1'
ENV MSAN_OPTIONS='abort_on_error=1'
CMD sleep 1

View File

@ -78,11 +78,16 @@ for e in root.findall('query'):
assert(len(test_queries) == len(is_short))
# If we're given a list of queries to run, check that it makes sense.
for i in args.queries_to_run or []:
if i < 0 or i >= len(test_queries):
print(f'There is no query no. {i} in this test, only [{0}-{len(test_queries) - 1}] are present')
exit(1)
# If we're only asked to print the queries, do that and exit
# If we're only asked to print the queries, do that and exit.
if args.print_queries:
for q in test_queries:
print(q)
for i in args.queries_to_run or range(0, len(test_queries)):
print(test_queries[i])
exit(0)
# Print short queries
@ -196,12 +201,7 @@ if args.max_queries:
queries_to_run = random.sample(range(0, len(test_queries)), min(len(test_queries), args.max_queries))
if args.queries_to_run:
# Run the specified queries, with some sanity check.
for i in args.queries_to_run:
if i < 0 or i >= len(test_queries):
print(f'There is no query no. "{i}" in this test, only [{0}-{len(test_queries) - 1}] are present')
exit(1)
# Run the specified queries.
queries_to_run = args.queries_to_run
# Run test queries.

View File

@ -42,7 +42,10 @@ function start()
# install test configs
/usr/share/clickhouse-test/config/install.sh
# for clickhouse-server (via service)
echo "ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000'" >> /etc/environment
# for clickhouse-client
export ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000'
start

View File

@ -33,6 +33,7 @@ Columns:
- `'ExceptionWhileProcessing' = 4` — Exception during the query execution.
- `event_date` ([Date](../../sql-reference/data-types/date.md)) — Query starting date.
- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Query starting time.
- `event_time_microseconds` ([DateTime](../../sql-reference/data-types/datetime.md)) — Query starting time with microseconds precision.
- `query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Start time of query execution.
- `query_start_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Start time of query execution with microsecond precision.
- `query_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Duration of query execution in milliseconds.
@ -84,54 +85,57 @@ Columns:
**Example**
``` sql
SELECT * FROM system.query_log LIMIT 1 FORMAT Vertical;
SELECT * FROM system.query_log LIMIT 1 \G
```
``` text
Row 1:
──────
type: QueryStart
event_date: 2020-05-13
event_time: 2020-05-13 14:02:28
query_start_time: 2020-05-13 14:02:28
query_duration_ms: 0
read_rows: 0
read_bytes: 0
written_rows: 0
written_bytes: 0
result_rows: 0
result_bytes: 0
memory_usage: 0
query: SELECT 1
exception_code: 0
exception:
stack_trace:
is_initial_query: 1
user: default
query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a
address: ::ffff:127.0.0.1
port: 57720
initial_user: default
initial_query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a
initial_address: ::ffff:127.0.0.1
initial_port: 57720
interface: 1
os_user: bayonet
client_hostname: clickhouse.ru-central1.internal
client_name: ClickHouse client
client_revision: 54434
client_version_major: 20
client_version_minor: 4
client_version_patch: 1
http_method: 0
http_user_agent:
quota_key:
revision: 54434
thread_ids: []
ProfileEvents.Names: []
ProfileEvents.Values: []
Settings.Names: ['use_uncompressed_cache','load_balancing','log_queries','max_memory_usage']
Settings.Values: ['0','random','1','10000000000']
type: QueryStart
event_date: 2020-09-11
event_time: 2020-09-11 10:08:17
event_time_microseconds: 2020-09-11 10:08:17.063321
query_start_time: 2020-09-11 10:08:17
query_start_time_microseconds: 2020-09-11 10:08:17.063321
query_duration_ms: 0
read_rows: 0
read_bytes: 0
written_rows: 0
written_bytes: 0
result_rows: 0
result_bytes: 0
memory_usage: 0
current_database: default
query: INSERT INTO test1 VALUES
exception_code: 0
exception:
stack_trace:
is_initial_query: 1
user: default
query_id: 50a320fd-85a8-49b8-8761-98a86bcbacef
address: ::ffff:127.0.0.1
port: 33452
initial_user: default
initial_query_id: 50a320fd-85a8-49b8-8761-98a86bcbacef
initial_address: ::ffff:127.0.0.1
initial_port: 33452
interface: 1
os_user: bharatnc
client_hostname: tower
client_name: ClickHouse
client_revision: 54437
client_version_major: 20
client_version_minor: 7
client_version_patch: 2
http_method: 0
http_user_agent:
quota_key:
revision: 54440
thread_ids: []
ProfileEvents.Names: []
ProfileEvents.Values: []
Settings.Names: ['use_uncompressed_cache','load_balancing','log_queries','max_memory_usage','allow_introspection_functions']
Settings.Values: ['0','random','1','10000000000','1']
```
**See Also**

View File

@ -15,6 +15,7 @@ Columns:
- `event_date` ([Date](../../sql-reference/data-types/date.md)) — The date when the thread has finished execution of the query.
- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — The date and time when the thread has finished execution of the query.
- `event_time_microsecinds` ([DateTime](../../sql-reference/data-types/datetime.md)) — The date and time when the thread has finished execution of the query with microseconds precision.
- `query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Start time of query execution.
- `query_start_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Start time of query execution with microsecond precision.
- `query_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Duration of query execution.
@ -63,50 +64,51 @@ Columns:
**Example**
``` sql
SELECT * FROM system.query_thread_log LIMIT 1 FORMAT Vertical
SELECT * FROM system.query_thread_log LIMIT 1 \G
```
``` text
Row 1:
──────
event_date: 2020-05-13
event_time: 2020-05-13 14:02:28
query_start_time: 2020-05-13 14:02:28
query_duration_ms: 0
read_rows: 1
read_bytes: 1
written_rows: 0
written_bytes: 0
memory_usage: 0
peak_memory_usage: 0
thread_name: QueryPipelineEx
thread_id: 28952
master_thread_id: 28924
query: SELECT 1
is_initial_query: 1
user: default
query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a
address: ::ffff:127.0.0.1
port: 57720
initial_user: default
initial_query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a
initial_address: ::ffff:127.0.0.1
initial_port: 57720
interface: 1
os_user: bayonet
client_hostname: clickhouse.ru-central1.internal
client_name: ClickHouse client
client_revision: 54434
client_version_major: 20
client_version_minor: 4
client_version_patch: 1
http_method: 0
http_user_agent:
quota_key:
revision: 54434
ProfileEvents.Names: ['ContextLock','RealTimeMicroseconds','UserTimeMicroseconds','OSCPUWaitMicroseconds','OSCPUVirtualTimeMicroseconds']
ProfileEvents.Values: [1,97,81,5,81]
...
event_date: 2020-09-11
event_time: 2020-09-11 10:08:17
event_time_microseconds: 2020-09-11 10:08:17.134042
query_start_time: 2020-09-11 10:08:17
query_start_time_microseconds: 2020-09-11 10:08:17.063150
query_duration_ms: 70
read_rows: 0
read_bytes: 0
written_rows: 1
written_bytes: 12
memory_usage: 4300844
peak_memory_usage: 4300844
thread_name: TCPHandler
thread_id: 638133
master_thread_id: 638133
query: INSERT INTO test1 VALUES
is_initial_query: 1
user: default
query_id: 50a320fd-85a8-49b8-8761-98a86bcbacef
address: ::ffff:127.0.0.1
port: 33452
initial_user: default
initial_query_id: 50a320fd-85a8-49b8-8761-98a86bcbacef
initial_address: ::ffff:127.0.0.1
initial_port: 33452
interface: 1
os_user: bharatnc
client_hostname: tower
client_name: ClickHouse
client_revision: 54437
client_version_major: 20
client_version_minor: 7
client_version_patch: 2
http_method: 0
http_user_agent:
quota_key:
revision: 54440
ProfileEvents.Names: ['Query','InsertQuery','FileOpen','WriteBufferFromFileDescriptorWrite','WriteBufferFromFileDescriptorWriteBytes','ReadCompressedBytes','CompressedReadBufferBlocks','CompressedReadBufferBytes','IOBufferAllocs','IOBufferAllocBytes','FunctionExecute','CreatedWriteBufferOrdinary','DiskWriteElapsedMicroseconds','NetworkReceiveElapsedMicroseconds','NetworkSendElapsedMicroseconds','InsertedRows','InsertedBytes','SelectedRows','SelectedBytes','MergeTreeDataWriterRows','MergeTreeDataWriterUncompressedBytes','MergeTreeDataWriterCompressedBytes','MergeTreeDataWriterBlocks','MergeTreeDataWriterBlocksAlreadySorted','ContextLock','RWLockAcquiredReadLocks','RealTimeMicroseconds','UserTimeMicroseconds','SoftPageFaults','OSCPUVirtualTimeMicroseconds','OSWriteBytes','OSReadChars','OSWriteChars']
ProfileEvents.Values: [1,1,11,11,591,148,3,71,29,6533808,1,11,72,18,47,1,12,1,12,1,12,189,1,1,10,2,70853,2748,49,2747,45056,422,1520]
```
**See Also**

View File

@ -6,6 +6,7 @@ Columns:
- `event_date` (Date) — Date of the entry.
- `event_time` (DateTime) — Time of the entry.
- `event_time_microseconds` (DateTime) — Time of the entry with microseconds precision.
- `microseconds` (UInt32) — Microseconds of the entry.
- `thread_name` (String) — Name of the thread from which the logging was done.
- `thread_id` (UInt64) — OS thread ID.
@ -25,4 +26,28 @@ Columns:
- `source_file` (LowCardinality(String)) — Source file from which the logging was done.
- `source_line` (UInt64) — Source line from which the logging was done.
**Example**
``` sql
SELECT * FROM system.text_log LIMIT 1 \G
```
``` text
Row 1:
──────
event_date: 2020-09-10
event_time: 2020-09-10 11:23:07
event_time_microseconds: 2020-09-10 11:23:07.871397
microseconds: 871397
thread_name: clickhouse-serv
thread_id: 564917
level: Information
query_id:
logger_name: DNSCacheUpdater
message: Update period 15 seconds
revision: 54440
source_file: /ClickHouse/src/Interpreters/DNSCacheUpdater.cpp; void DB::DNSCacheUpdater::start()
source_line: 45
```
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/text_log) <!--hide-->

View File

@ -12,6 +12,8 @@ Columns:
- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Timestamp of the sampling moment.
- `event_time_microseconds` ([DateTime](../../sql-reference/data-types/datetime.md)) — Timestamp of the sampling moment with microseconds precision.
- `timestamp_ns` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Timestamp of the sampling moment in nanoseconds.
- `revision` ([UInt32](../../sql-reference/data-types/int-uint.md)) — ClickHouse server build revision.
@ -38,13 +40,16 @@ SELECT * FROM system.trace_log LIMIT 1 \G
``` text
Row 1:
──────
event_date: 2019-11-15
event_time: 2019-11-15 15:09:38
revision: 54428
timer_type: Real
thread_number: 48
query_id: acc4d61f-5bd1-4a3e-bc91-2180be37c915
trace: [94222141367858,94222152240175,94222152325351,94222152329944,94222152330796,94222151449980,94222144088167,94222151682763,94222144088167,94222151682763,94222144088167,94222144058283,94222144059248,94222091840750,94222091842302,94222091831228,94222189631488,140509950166747,140509942945935]
event_date: 2020-09-10
event_time: 2020-09-10 11:23:09
event_time_microseconds: 2020-09-10 11:23:09.872924
timestamp_ns: 1599762189872924510
revision: 54440
trace_type: Memory
thread_id: 564963
query_id:
trace: [371912858,371912789,371798468,371799717,371801313,371790250,624462773,566365041,566440261,566445834,566460071,566459914,566459842,566459580,566459469,566459389,566459341,566455774,371993941,371988245,372158848,372187428,372187309,372187093,372185478,140222123165193,140222122205443]
size: 5244400
```
[Original article](https://clickhouse.tech/docs/en/operations/system_tables/trace_log) <!--hide-->

View File

@ -552,7 +552,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
#if defined(__linux__)
fmt::print("Setting capabilities for clickhouse binary. This is optional.\n");
std::string command = fmt::format("command setcap && setcap 'cap_net_admin,cap_ipc_lock,cap_sys_nice+ep' {}", main_bin_path.string());
std::string command = fmt::format("command -v setcap && setcap 'cap_net_admin,cap_ipc_lock,cap_sys_nice+ep' {}", main_bin_path.string());
fmt::print(" {}\n", command);
executeScript(command);
#endif

View File

@ -210,7 +210,7 @@ try
/// Maybe useless
if (config().has("macros"))
context->setMacros(std::make_unique<Macros>(config(), "macros"));
context->setMacros(std::make_unique<Macros>(config(), "macros", log));
/// Skip networking

View File

@ -534,7 +534,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
}
if (config().has("macros"))
global_context->setMacros(std::make_unique<Macros>(config(), "macros"));
global_context->setMacros(std::make_unique<Macros>(config(), "macros", log));
/// Initialize main config reloader.
std::string include_from_path = config().getString("include_from", "/etc/metrika.xml");
@ -559,7 +559,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
//setTextLog(global_context->getTextLog());
//buildLoggers(*config, logger());
global_context->setClustersConfig(config);
global_context->setMacros(std::make_unique<Macros>(*config, "macros"));
global_context->setMacros(std::make_unique<Macros>(*config, "macros", log));
global_context->setExternalAuthenticatorsConfig(*config);
/// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default)

View File

@ -2,6 +2,7 @@
#include <Common/Macros.h>
#include <Common/Exception.h>
#include <IO/WriteHelpers.h>
#include <common/logger_useful.h>
namespace DB
@ -12,19 +13,32 @@ namespace ErrorCodes
extern const int SYNTAX_ERROR;
}
Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String & root_key)
Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String & root_key, Poco::Logger * log)
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(root_key, keys);
for (const String & key : keys)
{
macros[key] = config.getString(root_key + "." + key);
if (key == "database" || key == "table" || key == "uuid")
{
if (log)
LOG_WARNING(log,
"Config file contains '{}' macro. This macro has special meaning "
"and it's explicit definition is not recommended. Implicit unfolding for "
"'database', 'table' and 'uuid' macros will be disabled.",
key);
enable_special_macros = false;
}
}
}
String Macros::expand(const String & s,
MacroExpansionInfo & info) const
{
/// Do not allow recursion if we expand only special macros, because it will be infinite recursion
assert(info.level == 0 || !info.expand_special_macros_only);
if (s.find('{') == String::npos)
return s;
@ -34,6 +48,10 @@ String Macros::expand(const String & s,
if (info.level >= 10)
throw Exception("Too deep recursion while expanding macros: '" + s + "'", ErrorCodes::SYNTAX_ERROR);
/// If config file contains explicit special macro, then we do not expand it in this mode.
if (!enable_special_macros && info.expand_special_macros_only)
return s;
String res;
size_t pos = 0;
while (true)
@ -59,15 +77,21 @@ String Macros::expand(const String & s,
auto it = macros.find(macro_name);
/// Prefer explicit macros over implicit.
if (it != macros.end())
if (it != macros.end() && !info.expand_special_macros_only)
res += it->second;
else if (macro_name == "database" && !info.database_name.empty())
res += info.database_name;
else if (macro_name == "table" && !info.table_name.empty())
res += info.table_name;
else if (macro_name == "database" && !info.table_id.database_name.empty())
{
res += info.table_id.database_name;
info.expanded_database = true;
}
else if (macro_name == "table" && !info.table_id.table_name.empty())
{
res += info.table_id.table_name;
info.expanded_table = true;
}
else if (macro_name == "uuid")
{
if (info.uuid == UUIDHelpers::Nil)
if (info.table_id.uuid == UUIDHelpers::Nil)
throw Exception("Macro 'uuid' and empty arguments of ReplicatedMergeTree "
"are supported only for ON CLUSTER queries with Atomic database engine",
ErrorCodes::SYNTAX_ERROR);
@ -76,12 +100,16 @@ String Macros::expand(const String & s,
/// It becomes impossible to check if {uuid} is contained inside some unknown macro.
if (info.level)
throw Exception("Macro 'uuid' should not be inside another macro", ErrorCodes::SYNTAX_ERROR);
res += toString(info.uuid);
res += toString(info.table_id.uuid);
info.expanded_uuid = true;
}
else if (info.ignore_unknown)
else if (info.ignore_unknown || info.expand_special_macros_only)
{
if (info.expand_special_macros_only)
res += '{';
res += macro_name;
if (info.expand_special_macros_only)
res += '}';
info.has_unknown = true;
}
else
@ -93,6 +121,9 @@ String Macros::expand(const String & s,
}
++info.level;
if (info.expand_special_macros_only)
return res;
return expand(res, info);
}
@ -113,9 +144,9 @@ String Macros::expand(const String & s) const
String Macros::expand(const String & s, const StorageID & table_id, bool allow_uuid) const
{
MacroExpansionInfo info;
info.database_name = table_id.database_name;
info.table_name = table_id.table_name;
info.uuid = allow_uuid ? table_id.uuid : UUIDHelpers::Nil;
info.table_id = table_id;
if (!allow_uuid)
info.table_id.uuid = UUIDHelpers::Nil;
return expand(s, info);
}

View File

@ -13,6 +13,7 @@ namespace Poco
{
class AbstractConfiguration;
}
class Logger;
}
@ -25,18 +26,19 @@ class Macros
{
public:
Macros() = default;
Macros(const Poco::Util::AbstractConfiguration & config, const String & key);
Macros(const Poco::Util::AbstractConfiguration & config, const String & key, Poco::Logger * log = nullptr);
struct MacroExpansionInfo
{
/// Settings
String database_name;
String table_name;
UUID uuid = UUIDHelpers::Nil;
StorageID table_id = StorageID::createEmpty();
bool ignore_unknown = false;
bool expand_special_macros_only = false;
/// Information about macro expansion
size_t level = 0;
bool expanded_database = false;
bool expanded_table = false;
bool expanded_uuid = false;
bool has_unknown = false;
};
@ -64,6 +66,7 @@ public:
private:
MacroMap macros;
bool enable_special_macros = true;
};

View File

@ -13,7 +13,6 @@ inline UInt64 clock_gettime_ns(clockid_t clock_type = CLOCK_MONOTONIC)
return UInt64(ts.tv_sec * 1000000000LL + ts.tv_nsec);
}
/** Differs from Poco::Stopwatch only by using 'clock_gettime' instead of 'gettimeofday',
* returns nanoseconds instead of microseconds, and also by other minor differencies.
*/

View File

@ -141,8 +141,14 @@ void TraceCollector::run()
if (trace_log)
{
UInt64 time = clock_gettime_ns(CLOCK_REALTIME);
TraceLogElement element{time_t(time / 1000000000), time, trace_type, thread_id, query_id, trace, size};
// time and time_in_microseconds are both being constructed from the same timespec so that the
// times will be equal upto the precision of a second.
struct timespec ts;
clock_gettime(CLOCK_REALTIME, &ts);
UInt64 time = UInt64(ts.tv_sec * 1000000000LL + ts.tv_nsec);
UInt64 time_in_microseconds = 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};
trace_log->add(element);
}
}

View File

@ -774,7 +774,7 @@ void DataTypeLowCardinality::deserializeTextQuoted(IColumn & column, ReadBuffer
void DataTypeLowCardinality::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
deserializeImpl(column, &IDataType::deserializeAsTextEscaped, istr, settings);
deserializeImpl(column, &IDataType::deserializeAsWholeText, istr, settings);
}
void DataTypeLowCardinality::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const

View File

@ -207,11 +207,13 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot move dictionary to other database");
StoragePtr table = getTableUnlocked(table_name, db_lock);
table->checkTableCanBeRenamed();
assert_can_move_mat_view(table);
StoragePtr other_table;
if (exchange)
{
other_table = other_db.getTableUnlocked(to_table_name, other_db_lock);
other_table->checkTableCanBeRenamed();
assert_can_move_mat_view(other_table);
}

View File

@ -888,7 +888,7 @@ void InterpreterCreateQuery::prepareOnClusterQuery(ASTCreateQuery & create, cons
{
String zk_path = create.storage->engine->arguments->children[0]->as<ASTLiteral>()->value.get<String>();
Macros::MacroExpansionInfo info;
info.uuid = create.uuid;
info.table_id.uuid = create.uuid;
info.ignore_unknown = true;
context.getMacros()->expand(zk_path, info);
if (!info.expanded_uuid)

View File

@ -39,6 +39,7 @@ Block QueryLogElement::createBlock()
{std::move(query_status_datatype), "type"},
{std::make_shared<DataTypeDate>(), "event_date"},
{std::make_shared<DataTypeDateTime>(), "event_time"},
{std::make_shared<DataTypeDateTime64>(6), "event_time_microseconds"},
{std::make_shared<DataTypeDateTime>(), "query_start_time"},
{std::make_shared<DataTypeDateTime64>(6), "query_start_time_microseconds"},
{std::make_shared<DataTypeUInt64>(), "query_duration_ms"},
@ -97,6 +98,7 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insert(type);
columns[i++]->insert(DateLUT::instance().toDayNum(event_time));
columns[i++]->insert(event_time);
columns[i++]->insert(event_time_microseconds);
columns[i++]->insert(query_start_time);
columns[i++]->insert(query_start_time_microseconds);
columns[i++]->insert(query_duration_ms);

View File

@ -30,6 +30,7 @@ struct QueryLogElement
/// Depending on the type of query and type of stage, not all the fields may be filled.
time_t event_time{};
UInt64 event_time_microseconds{};
time_t query_start_time{};
UInt64 query_start_time_microseconds{};
UInt64 query_duration_ms{};

View File

@ -23,6 +23,7 @@ Block QueryThreadLogElement::createBlock()
return {
{std::make_shared<DataTypeDate>(), "event_date"},
{std::make_shared<DataTypeDateTime>(), "event_time"},
{std::make_shared<DataTypeDateTime64>(6), "event_time_microseconds"},
{std::make_shared<DataTypeDateTime>(), "query_start_time"},
{std::make_shared<DataTypeDateTime64>(6), "query_start_time_microseconds"},
{std::make_shared<DataTypeUInt64>(), "query_duration_ms"},
@ -73,6 +74,7 @@ void QueryThreadLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insert(DateLUT::instance().toDayNum(event_time));
columns[i++]->insert(event_time);
columns[i++]->insert(event_time_microseconds);
columns[i++]->insert(query_start_time);
columns[i++]->insert(query_start_time_microseconds);
columns[i++]->insert(query_duration_ms);

View File

@ -16,6 +16,7 @@ namespace DB
struct QueryThreadLogElement
{
time_t event_time{};
UInt64 event_time_microseconds{};
/// When query was attached to current thread
time_t query_start_time{};
/// same as above but adds microsecond precision

View File

@ -2,6 +2,7 @@
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeString.h>
#include <Common/ClickHouseRevision.h>
@ -29,6 +30,7 @@ Block TextLogElement::createBlock()
{
{std::make_shared<DataTypeDate>(), "event_date"},
{std::make_shared<DataTypeDateTime>(), "event_time"},
{std::make_shared<DataTypeDateTime64>(6), "event_time_microseconds"},
{std::make_shared<DataTypeUInt32>(), "microseconds"},
{std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()), "thread_name"},
@ -52,6 +54,7 @@ void TextLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insert(DateLUT::instance().toDayNum(event_time));
columns[i++]->insert(event_time);
columns[i++]->insert(event_time_microseconds);
columns[i++]->insert(microseconds);
columns[i++]->insertData(thread_name.data(), thread_name.size());

View File

@ -9,6 +9,7 @@ using Poco::Message;
struct TextLogElement
{
time_t event_time{};
UInt64 event_time_microseconds{};
UInt32 microseconds;
String thread_name;

View File

@ -322,7 +322,14 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log)
{
QueryThreadLogElement elem;
elem.event_time = time(nullptr);
// construct current_time and current_time_microseconds using the same time point
// so that the two times will always be equal up to a precision of a second.
const auto now = std::chrono::system_clock::now();
auto current_time = time_in_seconds(now);
auto current_time_microseconds = time_in_microseconds(now);
elem.event_time = current_time;
elem.event_time_microseconds = current_time_microseconds;
elem.query_start_time = query_start_time;
elem.query_start_time_microseconds = query_start_time_microseconds;
elem.query_duration_ms = (getCurrentTimeNanoseconds() - query_start_time_nanoseconds) / 1000000U;

View File

@ -4,6 +4,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <Common/ClickHouseRevision.h>
@ -26,6 +27,7 @@ Block TraceLogElement::createBlock()
{
{std::make_shared<DataTypeDate>(), "event_date"},
{std::make_shared<DataTypeDateTime>(), "event_time"},
{std::make_shared<DataTypeDateTime64>(6), "event_time_microseconds"},
{std::make_shared<DataTypeUInt64>(), "timestamp_ns"},
{std::make_shared<DataTypeUInt32>(), "revision"},
{std::make_shared<TraceDataType>(trace_values), "trace_type"},
@ -42,6 +44,7 @@ void TraceLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insert(DateLUT::instance().toDayNum(event_time));
columns[i++]->insert(event_time);
columns[i++]->insert(event_time_microseconds);
columns[i++]->insert(timestamp_ns);
columns[i++]->insert(ClickHouseRevision::getVersionRevision());
columns[i++]->insert(static_cast<UInt8>(trace_type));

View File

@ -18,6 +18,7 @@ struct TraceLogElement
static const TraceDataType::Values trace_values;
time_t event_time{};
UInt64 event_time_microseconds{};
UInt64 timestamp_ns{};
TraceType trace_type{};
UInt64 thread_id{};

View File

@ -206,10 +206,11 @@ static void onExceptionBeforeStart(const String & query_for_logging, Context & c
elem.type = QueryLogElementType::EXCEPTION_BEFORE_START;
// all callers to onExceptionBeforeStart upstream construct the timespec for event_time and
// event_time_microseconds from the same timespec. So it can be assumed that both of these
// all callers to onExceptionBeforeStart method construct the timespec for event_time and
// event_time_microseconds from the same time point. So, it can be assumed that both of these
// times are equal upto the precision of a second.
elem.event_time = current_time;
elem.event_time_microseconds = current_time_microseconds;
elem.query_start_time = current_time;
elem.query_start_time_microseconds = current_time_microseconds;
@ -319,7 +320,9 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
logQuery(query_for_logging, context, internal);
if (!internal)
{
onExceptionBeforeStart(query_for_logging, context, current_time, current_time_microseconds, ast);
}
throw;
}
@ -484,6 +487,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
elem.type = QueryLogElementType::QUERY_START;
elem.event_time = current_time;
elem.event_time_microseconds = current_time_microseconds;
elem.query_start_time = current_time;
elem.query_start_time_microseconds = current_time_microseconds;
@ -554,8 +558,11 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
elem.type = QueryLogElementType::QUERY_FINISH;
elem.event_time = time(nullptr);
// construct event_time and event_time_microseconds using the same time point
// so that the two times will always be equal up to a precision of a second.
const auto time_now = std::chrono::system_clock::now();
elem.event_time = time_in_seconds(time_now);
elem.event_time_microseconds = time_in_microseconds(time_now);
status_info_to_query_log(elem, info, ast);
auto progress_callback = context.getProgressCallback();
@ -615,7 +622,12 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
elem.type = QueryLogElementType::EXCEPTION_WHILE_PROCESSING;
elem.event_time = time(nullptr);
// event_time and event_time_microseconds are being constructed from the same time point
// to ensure that both the times will be equal upto the precision of a second.
const auto time_now = std::chrono::system_clock::now();
elem.event_time = time_in_seconds(time_now);
elem.event_time_microseconds = time_in_microseconds(time_now);
elem.query_duration_ms = 1000 * (elem.event_time - elem.query_start_time);
elem.exception_code = getCurrentExceptionCode();
elem.exception = getCurrentExceptionMessage(false);

View File

@ -51,6 +51,8 @@ RegexpRowInputFormat::ColumnFormat RegexpRowInputFormat::stringToFormat(const St
return ColumnFormat::Csv;
if (format == "JSON")
return ColumnFormat::Json;
if (format == "Raw")
return ColumnFormat::Raw;
throw Exception("Unsupported column format \"" + format + "\".", ErrorCodes::BAD_ARGUMENTS);
}
@ -88,6 +90,12 @@ bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns)
else
type->deserializeAsTextJSON(*columns[index], field_buf, format_settings);
break;
case ColumnFormat::Raw:
if (parse_as_nullable)
read = DataTypeNullable::deserializeWholeText(*columns[index], field_buf, format_settings, type);
else
type->deserializeAsWholeText(*columns[index], field_buf, format_settings);
break;
default:
break;
}

View File

@ -337,6 +337,8 @@ public:
throw Exception("Truncate is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
virtual void checkTableCanBeRenamed() const {}
/** Rename the table.
* Renaming a name in a file with metadata, the name in the list of tables in the RAM, is done separately.
* In this function, you need to rename the directory with the data, if any.

View File

@ -412,26 +412,31 @@ static StoragePtr create(const StorageFactory::Arguments & args)
/// For Replicated.
String zookeeper_path;
String replica_name;
bool allow_renaming = true;
if (replicated)
{
bool has_arguments = arg_num + 2 <= arg_cnt;
bool has_valid_arguments = has_arguments && engine_args[arg_num]->as<ASTLiteral>() && engine_args[arg_num + 1]->as<ASTLiteral>();
ASTLiteral * ast_zk_path;
ASTLiteral * ast_replica_name;
if (has_valid_arguments)
{
const auto * ast = engine_args[arg_num]->as<ASTLiteral>();
if (ast && ast->value.getType() == Field::Types::String)
zookeeper_path = safeGet<String>(ast->value);
/// Get path and name from engine arguments
ast_zk_path = engine_args[arg_num]->as<ASTLiteral>();
if (ast_zk_path && ast_zk_path->value.getType() == Field::Types::String)
zookeeper_path = safeGet<String>(ast_zk_path->value);
else
throw Exception(
"Path in ZooKeeper must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def),
ErrorCodes::BAD_ARGUMENTS);
++arg_num;
ast = engine_args[arg_num]->as<ASTLiteral>();
if (ast && ast->value.getType() == Field::Types::String)
replica_name = safeGet<String>(ast->value);
ast_replica_name = engine_args[arg_num]->as<ASTLiteral>();
if (ast_replica_name && ast_replica_name->value.getType() == Field::Types::String)
replica_name = safeGet<String>(ast_replica_name->value);
else
throw Exception(
"Replica name must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::BAD_ARGUMENTS);
@ -444,9 +449,24 @@ static StoragePtr create(const StorageFactory::Arguments & args)
else if (is_extended_storage_def && !has_arguments)
{
/// Try use default values if arguments are not specified.
/// It works for ON CLUSTER queries when database engine is Atomic and there are {shard} and {replica} in config.
zookeeper_path = "/clickhouse/tables/{uuid}/{shard}";
replica_name = "{replica}"; /// TODO maybe use hostname if {replica} is not defined?
/// Note: {uuid} macro works for ON CLUSTER queries when database engine is Atomic.
zookeeper_path = args.context.getConfigRef().getString("default_replica_path", "/clickhouse/tables/{uuid}/{shard}");
/// TODO maybe use hostname if {replica} is not defined?
replica_name = args.context.getConfigRef().getString("default_replica_name", "{replica}");
/// Modify query, so default values will be written to metadata
assert(arg_num == 0);
ASTs old_args;
std::swap(engine_args, old_args);
auto path_arg = std::make_shared<ASTLiteral>(zookeeper_path);
auto name_arg = std::make_shared<ASTLiteral>(replica_name);
ast_zk_path = path_arg.get();
ast_replica_name = name_arg.get();
engine_args.emplace_back(std::move(path_arg));
engine_args.emplace_back(std::move(name_arg));
std::move(std::begin(old_args), std::end(old_args), std::back_inserter(engine_args));
arg_num = 2;
arg_cnt += 2;
}
else
throw Exception("Expected two string literal arguments: zookeper_path and replica_name", ErrorCodes::BAD_ARGUMENTS);
@ -454,8 +474,44 @@ static StoragePtr create(const StorageFactory::Arguments & args)
/// Allow implicit {uuid} macros only for zookeeper_path in ON CLUSTER queries
bool is_on_cluster = args.local_context.getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY;
bool allow_uuid_macro = is_on_cluster || args.query.attach;
zookeeper_path = args.context.getMacros()->expand(zookeeper_path, args.table_id, allow_uuid_macro);
replica_name = args.context.getMacros()->expand(replica_name, args.table_id, false);
/// Unfold {database} and {table} macro on table creation, so table can be renamed.
/// We also unfold {uuid} macro, so path will not be broken after moving table from Atomic to Ordinary database.
if (!args.attach)
{
Macros::MacroExpansionInfo info;
/// NOTE: it's not recursive
info.expand_special_macros_only = true;
info.table_id = args.table_id;
if (!allow_uuid_macro)
info.table_id.uuid = UUIDHelpers::Nil;
zookeeper_path = args.context.getMacros()->expand(zookeeper_path, info);
info.level = 0;
info.table_id.uuid = UUIDHelpers::Nil;
replica_name = args.context.getMacros()->expand(replica_name, info);
}
ast_zk_path->value = zookeeper_path;
ast_replica_name->value = replica_name;
/// Expand other macros (such as {shard} and {replica}). We do not expand them on previous step
/// to make possible copying metadata files between replicas.
Macros::MacroExpansionInfo info;
info.table_id = args.table_id;
if (!allow_uuid_macro)
info.table_id.uuid = UUIDHelpers::Nil;
zookeeper_path = args.context.getMacros()->expand(zookeeper_path, info);
info.level = 0;
info.table_id.uuid = UUIDHelpers::Nil;
replica_name = args.context.getMacros()->expand(replica_name, info);
/// We do not allow renaming table with these macros in metadata, because zookeeper_path will be broken after RENAME TABLE.
/// NOTE: it may happen if table was created by older version of ClickHouse (< 20.10) and macros was not unfolded on table creation
/// or if one of these macros is recursively expanded from some other macro.
if (info.expanded_database || info.expanded_table)
allow_renaming = false;
}
/// This merging param maybe used as part of sorting key
@ -706,7 +762,8 @@ static StoragePtr create(const StorageFactory::Arguments & args)
date_column_name,
merging_params,
std::move(storage_settings),
args.has_force_restore_data_flag);
args.has_force_restore_data_flag,
allow_renaming);
else
return StorageMergeTree::create(
args.table_id,

View File

@ -180,7 +180,16 @@ StoragePtr StorageFactory::get(
.has_force_restore_data_flag = has_force_restore_data_flag
};
return storages.at(name).creator_fn(arguments);
auto res = storages.at(name).creator_fn(arguments);
if (!empty_engine_args.empty())
{
/// Storage creator modified empty arguments list, so we should modify the query
assert(storage_def && storage_def->engine && !storage_def->engine->arguments);
storage_def->engine->arguments = std::make_shared<ASTExpressionList>();
storage_def->engine->children.push_back(storage_def->engine->arguments);
storage_def->engine->arguments->children = empty_engine_args;
}
return res;
}
StorageFactory & StorageFactory::instance()

View File

@ -178,7 +178,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
const String & date_column_name,
const MergingParams & merging_params_,
std::unique_ptr<MergeTreeSettings> settings_,
bool has_force_restore_data_flag)
bool has_force_restore_data_flag,
bool allow_renaming_)
: MergeTreeData(table_id_,
relative_data_path_,
metadata_,
@ -200,6 +201,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
, cleanup_thread(*this)
, part_check_thread(*this)
, restarting_thread(*this)
, allow_renaming(allow_renaming_)
{
queue_updating_task = global_context.getSchedulePool().createTask(
getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::queueUpdatingTask)", [this]{ queueUpdatingTask(); });
@ -4187,8 +4189,17 @@ void StorageReplicatedMergeTree::checkTableCanBeDropped() const
global_context.checkTableCanBeDropped(table_id.database_name, table_id.table_name, getTotalActiveSizeInBytes());
}
void StorageReplicatedMergeTree::checkTableCanBeRenamed() const
{
if (!allow_renaming)
throw Exception("Cannot rename Replicated table, because zookeeper_path contains implicit 'database' or 'table' macro. "
"We cannot rename path in ZooKeeper, so path may become inconsistent with table name. If you really want to rename table, "
"you should edit metadata file first and restart server or reattach the table.", ErrorCodes::NOT_IMPLEMENTED);
}
void StorageReplicatedMergeTree::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
{
checkTableCanBeRenamed();
MergeTreeData::rename(new_path_to_table_data, new_table_id);
/// Update table name in zookeeper

View File

@ -128,6 +128,8 @@ public:
void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override;
void checkTableCanBeRenamed() const override;
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
bool supportsIndexForIn() const override { return true; }
@ -304,6 +306,9 @@ private:
/// True if replica was created for existing table with fixed granularity
bool other_replicas_fixed_granularity = false;
/// Do not allow RENAME TABLE if zookeeper_path contains {database} or {table} macro
const bool allow_renaming;
template <class Func>
void foreachCommittedParts(const Func & func) const;
@ -571,7 +576,8 @@ protected:
const String & date_column_name,
const MergingParams & merging_params_,
std::unique_ptr<MergeTreeSettings> settings_,
bool has_force_restore_data_flag);
bool has_force_restore_data_flag,
bool allow_renaming_);
};

View File

@ -572,6 +572,18 @@
"clang-tidy": "disable",
"with_coverage": false
}
},
"Release": {
"required_build_properties": {
"compiler": "gcc-10",
"package_type": "deb",
"build_type": "relwithdebuginfo",
"sanitizer": "none",
"bundled": "bundled",
"splitted": "unsplitted",
"clang-tidy": "disable",
"with_coverage": false
}
}
}
}

View File

@ -3,5 +3,7 @@
<test>Hello, world!</test>
<shard>s1</shard>
<replica>r1</replica>
<default_path_test>/clickhouse/tables/{database}/{shard}/</default_path_test>
<default_name_test>table_{table}</default_name_test>
</macros>
</yandex>

View File

@ -330,10 +330,12 @@ def test_replicated_without_arguments(test_cluster):
assert "are supported only for ON CLUSTER queries with Atomic database engine" in \
instance.query_and_get_error("CREATE TABLE test_atomic.rmt (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n")
test_cluster.ddl_check_query(instance,
"CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n")
"CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree() ORDER BY n")
test_cluster.ddl_check_query(instance, "DROP TABLE test_atomic.rmt ON CLUSTER cluster")
test_cluster.ddl_check_query(instance,
"CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n")
"CREATE TABLE test_atomic.rmt UUID '12345678-0000-4000-8000-000000000001' ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n")
assert instance.query("SHOW CREATE test_atomic.rmt FORMAT TSVRaw") == \
"CREATE TABLE test_atomic.rmt\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree('/clickhouse/tables/12345678-0000-4000-8000-000000000001/{shard}', '{replica}')\nORDER BY n\nSETTINGS index_granularity = 8192\n"
test_cluster.ddl_check_query(instance, "RENAME TABLE test_atomic.rmt TO test_atomic.rmt_renamed ON CLUSTER cluster")
test_cluster.ddl_check_query(instance,
"CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{uuid}/{shard}', '{replica}') ORDER BY n")
@ -349,6 +351,8 @@ def test_replicated_without_arguments(test_cluster):
assert "are supported only for ON CLUSTER queries with Atomic database engine" in \
instance.query_and_get_error("CREATE TABLE test_ordinary.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree('/{shard}/{uuid}/', '{replica}') ORDER BY n")
test_cluster.ddl_check_query(instance, "CREATE TABLE test_ordinary.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree('/{shard}/{table}/', '{replica}') ORDER BY n")
assert instance.query("SHOW CREATE test_ordinary.rmt FORMAT TSVRaw") == \
"CREATE TABLE test_ordinary.rmt\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree('/{shard}/rmt/', '{replica}')\nORDER BY n\nSETTINGS index_granularity = 8192\n"
test_cluster.ddl_check_query(instance, "DROP DATABASE test_ordinary ON CLUSTER cluster")
test_cluster.pm_random_drops.push_rules(rules)

View File

@ -238,7 +238,9 @@ def test_mysql_federated(mysql_server, server_address):
node.query('''INSERT INTO mysql_federated.test VALUES (0), (1), (5)''', settings={"password": "123"})
def check_retryable_error_in_stderr(stderr):
return "Can't connect to local MySQL server through socket" in stderr or "MySQL server has gone away" in stderr
return ("Can't connect to local MySQL server through socket" in stderr
or "MySQL server has gone away" in stderr
or "Server shutdown in progress" in stderr)
code, (stdout, stderr) = mysql_server.exec_run('''
mysql

View File

@ -12,7 +12,7 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_r1(d Date, x UInt32, s Stri
${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_r2(d Date, x UInt32, s String, m MATERIALIZED x + 2) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00652/mutations', 'r2', d, intDiv(x, 10), 8192)"
# Test a mutation on empty table
${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE x = 1"
${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE x = 1 SETTINGS mutations_sync = 2"
# Insert some data
${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_r1(d, x, s) VALUES \
@ -34,6 +34,8 @@ ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE m = 3 SETTIN
${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_r1(d, x, s) VALUES \
('2000-01-01', 5, 'e'), ('2000-02-01', 5, 'e')"
${CLICKHOUSE_CLIENT} --query "SYSTEM SYNC REPLICA mutations_r2"
# Check that the table contains only the data that should not be deleted.
${CLICKHOUSE_CLIENT} --query="SELECT d, x, s, m FROM mutations_r2 ORDER BY d, x"
# Check the contents of the system.mutations table.

View File

@ -0,0 +1,4 @@
CREATE TABLE default.rmt\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01148/{shard}/default/rmt\', \'{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192
CREATE TABLE default.rmt1\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01148/{shard}/default/rmt\', \'{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192
CREATE TABLE default.rmt\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'{default_path_test}test_01148\', \'{default_name_test}\')\nORDER BY n\nSETTINGS index_granularity = 8192
CREATE TABLE default.rmt\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'{default_path_test}test_01148\', \'{default_name_test}\')\nORDER BY n\nSETTINGS index_granularity = 8192

View File

@ -0,0 +1,19 @@
DROP TABLE IF EXISTS rmt;
CREATE TABLE rmt (n UInt64, s String) ENGINE = ReplicatedMergeTree('/clickhouse/test_01148/{shard}/{database}/{table}', '{replica}') ORDER BY n;
SHOW CREATE TABLE rmt;
RENAME TABLE rmt TO rmt1;
DETACH TABLE rmt1;
ATTACH TABLE rmt1;
SHOW CREATE TABLE rmt1;
CREATE TABLE rmt (n UInt64, s String) ENGINE = ReplicatedMergeTree('{default_path_test}{uuid}', '{default_name_test}') ORDER BY n; -- { serverError 62 }
CREATE TABLE rmt (n UInt64, s String) ENGINE = ReplicatedMergeTree('{default_path_test}test_01148', '{default_name_test}') ORDER BY n;
SHOW CREATE TABLE rmt;
RENAME TABLE rmt TO rmt2; -- { serverError 48 }
DETACH TABLE rmt;
ATTACH TABLE rmt;
SHOW CREATE TABLE rmt;
DROP TABLE rmt;
DROP TABLE rmt1;

View File

@ -1,2 +1,10 @@
01473_metric_log_table_event_start_time_microseconds_test
ok
01473_trace_log_table_event_start_time_microseconds_test
ok
01473_query_log_table_event_start_time_microseconds_test
ok
01473_query_thread_log_table_event_start_time_microseconds_test
ok
01473_text_log_table_event_start_time_microseconds_test
ok

View File

@ -3,22 +3,55 @@
-- an integration test as those metrics take 60s by default to be updated.
-- Refer: tests/integration/test_asynchronous_metric_log_table.
set log_queries = 1;
SET log_queries = 1;
SET query_profiler_real_time_period_ns = 100000000;
-- a long enough query to trigger the query profiler and to record trace log
SELECT sleep(2) FORMAT Null;
SET query_profiler_real_time_period_ns = 1000000000;
SYSTEM FLUSH LOGS;
select '01473_metric_log_table_event_start_time_microseconds_test';
system flush logs;
SELECT '01473_metric_log_table_event_start_time_microseconds_test';
-- query assumes that the event_time field is accurate.
WITH (
(
SELECT event_time_microseconds
SELECT event_time_microseconds, event_time
FROM system.metric_log
ORDER BY event_time DESC
LIMIT 1
) AS time_with_microseconds,
(
SELECT event_time
FROM system.metric_log
) AS time
SELECT if(dateDiff('second', toDateTime(time.1), toDateTime(time.2)) = 0, 'ok', toString(time));
SELECT '01473_trace_log_table_event_start_time_microseconds_test';
WITH (
SELECT event_time_microseconds, event_time
FROM system.trace_log
ORDER BY event_time DESC
LIMIT 1
) AS time
SELECT if(dateDiff('second', toDateTime(time.1), toDateTime(time.2)) = 0, 'ok', toString(time));
SELECT '01473_query_log_table_event_start_time_microseconds_test';
WITH (
SELECT event_time_microseconds, event_time
FROM system.query_log
ORDER BY event_time DESC
LIMIT 1
) AS time)
SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail')
) AS time
SELECT if(dateDiff('second', toDateTime(time.1), toDateTime(time.2)) = 0, 'ok', toString(time));
SELECT '01473_query_thread_log_table_event_start_time_microseconds_test';
WITH (
SELECT event_time_microseconds, event_time
FROM system.query_thread_log
ORDER BY event_time DESC
LIMIT 1
) AS time
SELECT if(dateDiff('second', toDateTime(time.1), toDateTime(time.2)) = 0, 'ok', toString(time));
SELECT '01473_text_log_table_event_start_time_microseconds_test';
WITH (
SELECT event_time_microseconds, event_time
FROM system.query_thread_log
ORDER BY event_time DESC
LIMIT 1
) AS time
SELECT if(dateDiff('second', toDateTime(time.1), toDateTime(time.2)) = 0, 'ok', toString(time));

View File

@ -0,0 +1 @@
abc\\ Hello, world!

View File

@ -0,0 +1,17 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} -n --query "
DROP TABLE IF EXISTS t;
CREATE TABLE t (a String, b String) ENGINE = Memory;
"
${CLICKHOUSE_CLIENT} --format_regexp_escaping_rule 'Raw' --format_regexp '^(.+?) separator (.+?)$' --query '
INSERT INTO t FORMAT Regexp abc\ separator Hello, world!'
${CLICKHOUSE_CLIENT} -n --query "
SELECT * FROM t;
DROP TABLE t;
"

View File

@ -0,0 +1 @@
abc\\ Hello, world!

View File

@ -0,0 +1,17 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} -n --query "
DROP TABLE IF EXISTS t;
CREATE TABLE t (a String, b LowCardinality(Nullable(String))) ENGINE = Memory;
"
${CLICKHOUSE_CLIENT} --format_regexp_escaping_rule 'Raw' --format_regexp '^(.+?) separator (.+?)$' --query '
INSERT INTO t FORMAT Regexp abc\ separator Hello, world!'
${CLICKHOUSE_CLIENT} -n --query "
SELECT * FROM t;
DROP TABLE t;
"

View File

@ -17,6 +17,7 @@
"00151_replace_partition_with_different_granularity",
"00157_cache_dictionary",
"01193_metadata_loading",
"01473_event_time_microseconds",
"01474_executable_dictionary" /// informational stderr from sanitizer at start
],
"address-sanitizer": [
@ -25,6 +26,7 @@
"memory_profiler",
"odbc_roundtrip",
"01103_check_cpu_instructions_at_startup",
"01473_event_time_microseconds",
"01193_metadata_loading"
],
"ub-sanitizer": [
@ -33,6 +35,7 @@
"memory_profiler",
"01103_check_cpu_instructions_at_startup",
"00900_orc_load",
"01473_event_time_microseconds",
"01193_metadata_loading"
],
"memory-sanitizer": [
@ -43,6 +46,7 @@
"01086_odbc_roundtrip", /// can't pass because odbc libraries are not instrumented
"00877_memory_limit_for_new_delete", /// memory limits don't work correctly under msan because it replaces malloc/free
"01114_mysql_database_engine_segfault", /// it fails in _nss_files_parse_servent while using NSS from GLibc to authenticate (need to get rid of it)
"01473_event_time_microseconds",
"01193_metadata_loading"
],
"debug-build": [
@ -57,6 +61,7 @@
"01037_polygon_dicts_",
"hyperscan",
"01193_metadata_loading",
"01473_event_time_microseconds",
"01396_inactive_replica_cleanup_nodes"
],
"unbundled-build": [