Merge branch 'master' into rename-window-functions

This commit is contained in:
mergify[bot] 2021-12-07 21:04:53 +00:00 committed by GitHub
commit ed0c36bf32
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
68 changed files with 3453 additions and 588 deletions

View File

@ -149,6 +149,10 @@ if (ENABLE_FUZZING)
set (ENABLE_JEMALLOC 0)
set (ENABLE_CHECK_HEAVY_BUILDS 1)
set (GLIBC_COMPATIBILITY OFF)
# For codegen_select_fuzzer
set (ENABLE_PROTOBUF 1)
set (USE_INTERNAL_PROTOBUF_LIBRARY 1)
endif()
# Global libraries

View File

@ -48,7 +48,10 @@ struct StringRef
std::string toString() const { return std::string(data, size); }
explicit operator std::string() const { return toString(); }
constexpr explicit operator std::string_view() const { return {data, size}; }
std::string_view toView() const { return std::string_view(data, size); }
constexpr explicit operator std::string_view() const { return std::string_view(data, size); }
};
/// Here constexpr doesn't implicate inline, see https://www.viva64.com/en/w/v1043/

View File

@ -31,5 +31,6 @@ do
mv "$FUZZER_PATH" /output/fuzzers
done
tar -zcvf /output/fuzzers.tar.gz /output/fuzzers
rm -rf /output/fuzzers

View File

@ -55,9 +55,41 @@ function configure()
echo "<clickhouse><asynchronous_metrics_update_period_s>1</asynchronous_metrics_update_period_s></clickhouse>" \
> /etc/clickhouse-server/config.d/asynchronous_metrics_update_period_s.xml
local total_mem
total_mem=$(awk '/MemTotal/ { print $(NF-1) }' /proc/meminfo) # KiB
total_mem=$(( total_mem*1024 )) # bytes
# Set maximum memory usage as half of total memory (less chance of OOM).
echo "<clickhouse><max_server_memory_usage_to_ram_ratio>0.5</max_server_memory_usage_to_ram_ratio></clickhouse>" \
> /etc/clickhouse-server/config.d/max_server_memory_usage_to_ram_ratio.xml
#
# But not via max_server_memory_usage but via max_memory_usage_for_user,
# so that we can override this setting and execute service queries, like:
# - hung check
# - show/drop database
# - ...
#
# So max_memory_usage_for_user will be a soft limit, and
# max_server_memory_usage will be hard limit, and queries that should be
# executed regardless memory limits will use max_memory_usage_for_user=0,
# instead of relying on max_untracked_memory
local max_server_mem
max_server_mem=$((total_mem*75/100)) # 75%
echo "Setting max_server_memory_usage=$max_server_mem"
cat > /etc/clickhouse-server/config.d/max_server_memory_usage.xml <<EOL
<clickhouse>
<max_server_memory_usage>${max_server_mem}</max_server_memory_usage>
</clickhouse>
EOL
local max_users_mem
max_users_mem=$((total_mem*50/100)) # 50%
echo "Setting max_memory_usage_for_user=$max_users_mem"
cat > /etc/clickhouse-server/users.d/max_memory_usage_for_user.xml <<EOL
<clickhouse>
<profiles>
<default>
<max_memory_usage_for_user>${max_users_mem}</max_memory_usage_for_user>
</default>
</profiles>
</clickhouse>
EOL
}
function stop()

View File

@ -75,6 +75,9 @@ def call_with_retry(query, timeout=30, retry_count=5):
else:
break
def make_query_command(query):
return f"""clickhouse client -q "{query}" --max_untracked_memory=1Gi --memory_profiler_step=1Gi --max_memory_usage_for_user=0"""
def prepare_for_hung_check(drop_databases):
# FIXME this function should not exist, but...
@ -88,40 +91,41 @@ def prepare_for_hung_check(drop_databases):
logging.info("Will terminate gdb (if any)")
call_with_retry("kill -TERM $(pidof gdb)")
# Some tests set too low memory limit for default user and forget to reset in back.
# It may cause SYSTEM queries to fail, let's disable memory limit.
call_with_retry("clickhouse client --max_memory_usage_for_user=0 -q 'SELECT 1 FORMAT Null'")
call_with_retry(make_query_command('SELECT 1 FORMAT Null'))
# Some tests execute SYSTEM STOP MERGES or similar queries.
# It may cause some ALTERs to hang.
# Possibly we should fix tests and forbid to use such queries without specifying table.
call_with_retry("clickhouse client -q 'SYSTEM START MERGES'")
call_with_retry("clickhouse client -q 'SYSTEM START DISTRIBUTED SENDS'")
call_with_retry("clickhouse client -q 'SYSTEM START TTL MERGES'")
call_with_retry("clickhouse client -q 'SYSTEM START MOVES'")
call_with_retry("clickhouse client -q 'SYSTEM START FETCHES'")
call_with_retry("clickhouse client -q 'SYSTEM START REPLICATED SENDS'")
call_with_retry("clickhouse client -q 'SYSTEM START REPLICATION QUEUES'")
call_with_retry(make_query_command('SYSTEM START MERGES'))
call_with_retry(make_query_command('SYSTEM START DISTRIBUTED SENDS'))
call_with_retry(make_query_command('SYSTEM START TTL MERGES'))
call_with_retry(make_query_command('SYSTEM START MOVES'))
call_with_retry(make_query_command('SYSTEM START FETCHES'))
call_with_retry(make_query_command('SYSTEM START REPLICATED SENDS'))
call_with_retry(make_query_command('SYSTEM START REPLICATION QUEUES'))
call_with_retry(make_query_command('SYSTEM DROP MARK CACHE'))
# Issue #21004, live views are experimental, so let's just suppress it
call_with_retry("""clickhouse client -q "KILL QUERY WHERE upper(query) LIKE 'WATCH %'" """)
call_with_retry(make_query_command("KILL QUERY WHERE upper(query) LIKE 'WATCH %'"))
# Kill other queries which known to be slow
# It's query from 01232_preparing_sets_race_condition_long, it may take up to 1000 seconds in slow builds
call_with_retry("""clickhouse client -q "KILL QUERY WHERE query LIKE 'insert into tableB select %'" """)
call_with_retry(make_query_command("KILL QUERY WHERE query LIKE 'insert into tableB select %'"))
# Long query from 00084_external_agregation
call_with_retry("""clickhouse client -q "KILL QUERY WHERE query LIKE 'SELECT URL, uniq(SearchPhrase) AS u FROM test.hits GROUP BY URL ORDER BY u %'" """)
call_with_retry(make_query_command("KILL QUERY WHERE query LIKE 'SELECT URL, uniq(SearchPhrase) AS u FROM test.hits GROUP BY URL ORDER BY u %'"))
if drop_databases:
for i in range(5):
try:
# Here we try to drop all databases in async mode. If some queries really hung, than drop will hung too.
# Otherwise we will get rid of queries which wait for background pool. It can take a long time on slow builds (more than 900 seconds).
databases = check_output('clickhouse client -q "SHOW DATABASES"', shell=True, timeout=30).decode('utf-8').strip().split()
#
# Also specify max_untracked_memory to allow 1GiB of memory to overcommit.
databases = check_output(make_query_command('SHOW DATABASES'), shell=True, timeout=30).decode('utf-8').strip().split()
for db in databases:
if db == "system":
continue
command = f'clickhouse client -q "DROP DATABASE {db}"'
command = make_query_command(f'DROP DATABASE {db}')
# we don't wait for drop
Popen(command, shell=True)
break
@ -133,9 +137,15 @@ def prepare_for_hung_check(drop_databases):
# Wait for last queries to finish if any, not longer than 300 seconds
call("""clickhouse client -q "select sleepEachRow((
select maxOrDefault(300 - elapsed) + 1 from system.processes where query not like '%from system.processes%' and elapsed < 300
) / 300) from numbers(300) format Null" """, shell=True, stderr=STDOUT, timeout=330)
call(make_query_command("""
select sleepEachRow((
select maxOrDefault(300 - elapsed) + 1
from system.processes
where query not like '%from system.processes%' and elapsed < 300
) / 300)
from numbers(300)
format Null
"""), shell=True, stderr=STDOUT, timeout=330)
# Even if all clickhouse-test processes are finished, there are probably some sh scripts,
# which still run some new queries. Let's ignore them.
@ -188,7 +198,24 @@ if __name__ == "__main__":
if args.hung_check:
have_long_running_queries = prepare_for_hung_check(args.drop_databases)
logging.info("Checking if some queries hung")
cmd = "{} {} {}".format(args.test_cmd, "--hung-check", "00001_select_1")
cmd = ' '.join([args.test_cmd,
# Do not track memory allocations up to 1Gi,
# this will allow to ignore server memory limit (max_server_memory_usage) for this query.
#
# NOTE: memory_profiler_step should be also adjusted, because:
#
# untracked_memory_limit = min(settings.max_untracked_memory, settings.memory_profiler_step)
#
# NOTE: that if there will be queries with GROUP BY, this trick
# will not work due to CurrentMemoryTracker::check() from
# Aggregator code.
# But right now it should work, since neither hung check, nor 00001_select_1 has GROUP BY.
"--client-option", "max_untracked_memory=1Gi",
"--client-option", "max_memory_usage_for_user=0",
"--client-option", "memory_profiler_step=1Gi",
"--hung-check",
"00001_select_1"
])
res = call(cmd, shell=True, stderr=STDOUT)
hung_check_status = "No queries hung\tOK\n"
if res != 0 and have_long_running_queries:

View File

@ -992,9 +992,16 @@ log_queries_min_type='EXCEPTION_WHILE_PROCESSING'
Setting up query threads logging.
Queries threads run by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) server configuration parameter.
Query threads log into [system.query_thread_log](../../operations/system-tables/query_thread_log.md) table. This setting have effect only when [log_queries](#settings-log-queries) is true. Queries threads run by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) server configuration parameter.
Example:
Possible values:
- 0 — Disabled.
- 1 — Enabled.
Default value: `1`.
**Example**
``` text
log_query_threads=1

View File

@ -4,8 +4,8 @@ Contains information about the dependent views executed when running a query, fo
To start logging:
1. Configure parameters in the [query_views_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_views_log) section.
2. Set [log_query_views](../../operations/settings/settings.md#settings-log-query-views) to 1.
1. Configure parameters in the [query_views_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_views_log) section.
2. Set [log_query_views](../../operations/settings/settings.md#settings-log-query-views) to 1.
The flushing period of data is set in `flush_interval_milliseconds` parameter of the [query_views_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_views_log) server settings section. To force flushing, use the [SYSTEM FLUSH LOGS](../../sql-reference/statements/system.md#query_language-system-flush_logs) query.

View File

@ -0,0 +1,114 @@
---
toc_priority: 68
toc_title: Window View
---
# Window View Functions {#window-view-functions}
Window functions indicate the lower and upper window bound of records in WindowView. The functions for working with WindowView are listed below.
## tumble {#window-view-functions-tumble}
A tumbling time window assigns records to non-overlapping, continuous windows with a fixed duration (interval).
``` sql
tumble(time_attr, interval [, timezone])
```
**Arguments**
- `time_attr` - Date and time. [DateTime](../../sql-reference/data-types/datetime.md) data type.
- `interval` - Window interval in [Interval](../../sql-reference/data-types/special-data-types/interval.md) data type.
- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional).
**Returned values**
- The lower and upper bound of the tumble window.
Type: `Tuple(DateTime, DateTime)`
**Example**
Query:
``` sql
SELECT tumble(now(), toIntervalDay('1'))
```
Result:
``` text
┌─tumble(now(), toIntervalDay('1'))─────────────┐
│ ['2020-01-01 00:00:00','2020-01-02 00:00:00'] │
└───────────────────────────────────────────────┘
```
## hop {#window-view-functions-hop}
A hopping time window has a fixed duration (`window_interval`) and hops by a specified hop interval (`hop_interval`). If the `hop_interval` is smaller than the `window_interval`, hopping windows are overlapping. Thus, records can be assigned to multiple windows.
``` sql
hop(time_attr, hop_interval, window_interval [, timezone])
```
**Arguments**
- `time_attr` - Date and time. [DateTime](../../sql-reference/data-types/datetime.md) data type.
- `hop_interval` - Hop interval in [Interval](../../sql-reference/data-types/special-data-types/interval.md) data type. Should be a positive number.
- `window_interval` - Window interval in [Interval](../../sql-reference/data-types/special-data-types/interval.md) data type. Should be a positive number.
- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional).
**Returned values**
- The lower and upper bound of the hop window. Since hop windows are
overlapped, the function only returns the bound of the **first** window when
hop function is used **without** `WINDOW VIEW`.
Type: `Tuple(DateTime, DateTime)`
**Example**
Query:
``` sql
SELECT hop(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND)
```
Result:
``` text
┌─hop(now(), toIntervalSecond('1'), toIntervalSecond('2'))──┐
│ ('2020-01-14 16:58:22','2020-01-14 16:58:24') │
└───────────────────────────────────────────────────────────┘
```
## tumbleStart {#window-view-functions-tumblestart}
Indicate the lower bound of a tumble function.
``` sql
tumbleStart(time_attr, interval [, timezone]);
```
## tumbleEnd {#window-view-functions-tumbleend}
Indicate the upper bound of a tumble function.
``` sql
tumbleEnd(time_attr, interval [, timezone]);
```
## hopStart {#window-view-functions-hopstart}
Indicate the lower bound of a hop function.
``` sql
hopStart(time_attr, hop_interval, window_interval [, timezone]);
```
## hopEnd {#window-view-functions-hopend}
Indicate the upper bound of a hop function.
``` sql
hopEnd(time_attr, hop_interval, window_interval [, timezone]);
```

View File

@ -5,7 +5,7 @@ toc_title: VIEW
# CREATE VIEW {#create-view}
Creates a new view. Views can be [normal](#normal), [materialized](#materialized) and [live](#live-view) (the latter is an experimental feature).
Creates a new view. Views can be [normal](#normal), [materialized](#materialized), [live](#live-view), and [window](#window-view) (live view and window view are experimental features).
## Normal View {#normal}
@ -243,3 +243,119 @@ Most common uses of live view tables include:
**See Also**
- [ALTER LIVE VIEW](../alter/view.md#alter-live-view)
## Window View [Experimental] {#window-view}
!!! important "Important"
This is an experimental feature that may change in backwards-incompatible ways in the future releases.
Enable usage of window views and `WATCH` query using [allow_experimental_window_view](../../../operations/settings/settings.md#allow-experimental-window-view) setting. Input the command `set allow_experimental_window_view = 1`.
``` sql
CREATE WINDOW VIEW [IF NOT EXISTS] [db.]table_name [TO [db.]table_name] [ENGINE = engine] [WATERMARK = strategy] [ALLOWED_LATENESS = interval_function] AS SELECT ... GROUP BY window_view_function
```
Window view can aggregate data by time window and output the results when the window is ready to fire. It stores the partial aggregation results in an inner(or specified) table and can push the processing result to a specified table or push notifications using the WATCH query.
Creating a window view is similar to creating `MATERIALIZED VIEW`. Window view needs an inner storage engine to store intermediate data. The inner storage will use `AggregatingMergeTree` as the default engine.
### Window View Functions {#window-view-windowviewfunctions}
[Window view functions](../../functions/window-view-functions.md) are used to indicate the lower and upper window bound of records. The window view needs to be used with a window view function.
### TIME ATTRIBUTES {#window-view-timeattributes}
Window view supports **processing time** and **event time** process.
**Processing time** allows window view to produce results based on the local machine's time and is used by default. It is the most straightforward notion of time but does not provide determinism. The processing time attribute can be defined by setting the `time_attr` of the window view function to a table column or using the function `now()`. The following query creates a window view with processing time.
``` sql
CREATE WINDOW VIEW wv AS SELECT count(number), tumbleStart(w_id) as w_start from date GROUP BY tumble(now(), INTERVAL '5' SECOND) as w_id
```
**Event time** is the time that each individual event occurred on its producing device. This time is typically embedded within the records when it is generated. Event time processing allows for consistent results even in case of out-of-order events or late events. Window view supports event time processing by using `WATERMARK` syntax.
Window view provides three watermark strategies.
* `STRICTLY_ASCENDING`: Emits a watermark of the maximum observed timestamp so far. Rows that have a timestamp smaller to the max timestamp are not late.
* `ASCENDING`: Emits a watermark of the maximum observed timestamp so far minus 1. Rows that have a timestamp equal and smaller to the max timestamp are not late.
* `BOUNDED`: WATERMARK=INTERVAL. Emits watermarks, which are the maximum observed timestamp minus the specified delay.
The following queries are examples of creating a window view with `WATERMARK`.
``` sql
CREATE WINDOW VIEW wv WATERMARK=STRICTLY_ASCENDING AS SELECT count(number) FROM date GROUP BY tumble(timestamp, INTERVAL '5' SECOND);
CREATE WINDOW VIEW wv WATERMARK=ASCENDING AS SELECT count(number) FROM date GROUP BY tumble(timestamp, INTERVAL '5' SECOND);
CREATE WINDOW VIEW wv WATERMARK=INTERVAL '3' SECOND AS SELECT count(number) FROM date GROUP BY tumble(timestamp, INTERVAL '5' SECOND);
```
By default, the window will be fired when the watermark comes, and elements that arrived behind the watermark will be dropped. Window view supports late event processing by setting `ALLOWED_LATENESS=INTERVAL`. An example of lateness handling is:
``` sql
CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=ASCENDING ALLOWED_LATENESS=INTERVAL '2' SECOND AS SELECT count(a) AS count, tumbleEnd(wid) AS w_end FROM test.mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND) AS wid;
```
Note that elements emitted by a late firing should be treated as updated results of a previous computation. Instead of firing at the end of windows, the window view will fire immediately when the late event arrives. Thus, it will result in multiple outputs for the same window. Users need to take these duplicated results into account or deduplicate them.
### Monitoring New Windows{#window-view-monitoring}
Window view supports the `WATCH` query to constantly append the processing results to the console or use `TO` syntax to output the results to a table.
``` sql
WATCH [db.]name [LIMIT n]
```
`WATCH` query acts similar as in `LIVE VIEW`. A `LIMIT` can be specified to set the number of updates to receive before terminating the query.
### Settings {#window-view-settings}
- `window_view_clean_interval`: The clean interval of window view in seconds to free outdated data. The system will retain the windows that have not been fully triggered according to the system time or `WATERMARK` configuration, and the other data will be deleted.
- `window_view_heartbeat_interval`: The heartbeat interval in seconds to indicate the watch query is alive.
### Example {#window-view-example}
Suppose we need to count the number of click logs per 10 seconds in a log table called `data`, and its table structure is:
``` sql
CREATE TABLE data ( `id` UInt64, `timestamp` DateTime) ENGINE = Memory;
```
First, we create a window view with tumble window of 10 seconds interval:
``` sql
CREATE WINDOW VIEW wv as select count(id), tumbleStart(w_id) as window_start from data group by tumble(timestamp, INTERVAL '10' SECOND) as w_id
```
Then, we use the `WATCH` query to get the results.
``` sql
WATCH wv
```
When logs are inserted into table `data`,
``` sql
INSERT INTO data VALUES(1,now())
```
The `WATCH` query should print the results as follows:
``` text
┌─count(id)─┬────────window_start─┐
│ 1 │ 2020-01-14 16:56:40 │
└───────────┴─────────────────────┘
```
Alternatively, we can attach the output to another table using `TO` syntax.
``` sql
CREATE WINDOW VIEW wv TO dst AS SELECT count(id), tumbleStart(w_id) as window_start FROM data GROUP BY tumble(timestamp, INTERVAL '10' SECOND) as w_id
```
Additional examples can be found among stateful tests of ClickHouse (they are named `*window_view*` there).
### Window View Usage {#window-view-usage}
The window view is useful in the following scenarios:
* **Monitoring**: Aggregate and calculate the metrics logs by time, and output the results to a target table. The dashboard can use the target table as a source table.
* **Analyzing**: Automatically aggregate and preprocess data in the time window. This can be useful when analyzing a large number of logs. The preprocessing eliminates repeated calculations in multiple queries and reduces query latency.

View File

@ -999,14 +999,14 @@ ClickHouse проверяет условия для `min_part_size` и `min_part
Настройки логирования информации о зависимых представлениях (materialized, live и т.п.) в запросах принятых с настройкой [log_query_views=1](../../operations/settings/settings.md#settings-log-query-views).
Запросы сохраняются в таблицу system.query_views_log. Вы можете изменить название этой таблицы в параметре `table` (см. ниже).
Запросы логируются в таблице [system.query_views_log](../../operations/system-tables/query_views_log.md#system_tables-query_views_log). Вы можете изменить название этой таблицы в параметре `table` (см. ниже).
При настройке логирования используются следующие параметры:
- `database` имя базы данных.
- `table` имя таблицы куда будут записываться использованные представления.
- `partition_by` — устанавливает [произвольный ключ партиционирования](../../engines/table-engines/mergetree-family/custom-partitioning-key.md). Нельзя использовать если используется `engine`
- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`.
- `table` имя системной таблицы, где будут логироваться запросы.
- `partition_by` — устанавливает [произвольный ключ партиционирования](../../engines/table-engines/mergetree-family/custom-partitioning-key.md). Нельзя использовать, если задан параметр `engine`.
- `engine` устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать, если задан параметр `partition_by`.
- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу.
Если таблица не существует, то ClickHouse создаст её. Если структура журнала запросов изменилась при обновлении сервера ClickHouse, то таблица со старой структурой переименовывается, а новая таблица создается автоматически.

View File

@ -912,11 +912,18 @@ log_queries_min_type='EXCEPTION_WHILE_PROCESSING'
## log_query_threads {#settings-log-query-threads}
Установка логирования информации о потоках выполнения запроса.
Управляет логированием информации о потоках выполнения запросов.
Лог информации о потоках выполнения запросов, переданных в ClickHouse с этой установкой, записывается согласно правилам конфигурационного параметра сервера [query_thread_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log).
Информация о потоках выполнения запросов сохраняется в системной таблице [system.query_thread_log](../../operations/system-tables/query_thread_log.md). Работает только в том случае, если включена настройка [log_queries](#settings-log-queries). Лог информации о потоках выполнения запросов, переданных в ClickHouse с этой установкой, записывается согласно правилам конфигурационного параметра сервера [query_thread_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log).
Пример:
Возможные значения:
- 0 — отключено.
- 1 — включено.
Значение по умолчанию: `1`.
**Пример**
``` text
log_query_threads=1

View File

@ -55,6 +55,7 @@ ClickHouse не удаляет данные из таблица автомати
- `query_kind` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) — тип запроса.
- `databases` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — имена баз данных, присутствующих в запросе.
- `tables` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — имена таблиц, присутствующих в запросе.
- `views` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — имена представлений (материализованные или live), которые представленны в запросе.
- `columns` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — имена столбцов, присутствующих в запросе.
- `projections` ([String](../../sql-reference/data-types/string.md)) — имена проекций, использованных при выполнении запроса.
- `exception_code` ([Int32](../../sql-reference/data-types/int-uint.md)) — код исключения.

View File

@ -112,5 +112,5 @@ ProfileEvents: {'Query':1,'SelectQuery':1,'ReadCompressedBytes':36,'Compr
**Смотрите также**
- [system.query_log](../../operations/system-tables/query_log.md#system_tables-query_log) — описание системной таблицы `query_log`, которая содержит общую информацию о выполненных запросах.
- [system.query_log](../../operations/system-tables/query_log.md#system_tables-query_log) — описание системной таблицы `query_log`, которая содержит общую информацию о выполненных запросах.
- [system.query_views_log](../../operations/system-tables/query_views_log.md#system_tables-query_views_log) — описание системной таблицы `query_views_log`, которая содержит информацию о всех представлениях, участвующих в выполненных запросах.

View File

@ -1 +0,0 @@
../../../en/operations/system-tables/query_views_log.md

View File

@ -0,0 +1,84 @@
# system.query_views_log {#system_tables-query_views_log}
Содержит информацию о зависимых представлениях, выполняемых при выполнении запроса, например, тип представления или время выполнения.
Чтобы начать ведение журнала:
1. Настройте параметры в разделе [query_views_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_views_log).
2. Включите настройку [log_query_views=1](../../operations/settings/settings.md#settings-log-query-views).
Период сброса данных из буфера в памяти задается в параметре `flush_interval_milliseconds` в разделе настроек сервера [query_views_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_views_log ). Для принудительного сброса используйте запрос [SYSTEM FLUSH LOGS](../../sql-reference/statements/system.md#query_language-system-flush_logs).
ClickHouse не удаляет данные из таблицы автоматически. Подробнее смотрите раздел [Системные таблицы](../../operations/system-tables/index.md#system-tables-introduction).
Чтобы уменьшить количество запросов, регистрируемых в таблице `query_views_log`, вы можете включить настройку [log_queries_probability](../../operations/settings/settings.md#log-queries-probability).
Столбцы:
- `event_date` ([Date](../../sql-reference/data-types/date.md)) — дата, когда произошло последнее событие с представлением.
- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — дата и время завершения выполнения представления.
- `event_time_microseconds` ([DateTime](../../sql-reference/data-types/datetime.md)) — дата и время завершения выполнения представления с точностью до микросекунд.
- `view_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — продолжительность выполнения представления (сумма его этапов) в миллисекундах.
- `initial_query_id` ([String](../../sql-reference/data-types/string.md)) — идентификатор начального запроса (при распределённом выполнении запроса).
- `view_name` ([String](../../sql-reference/data-types/string.md)) — имя представления.
- `view_uuid` ([UUID](../../sql-reference/data-types/uuid.md)) — UUID представления.
- `view_type` ([Enum8](../../sql-reference/data-types/enum.md)) — тип представления. Возможные значения:
- `'Default' = 1` — [обычные представления](../../sql-reference/statements/create/view.md#normal). Не должно появляться в этом журнале.
- `'Materialized' = 2` — [материализованные представления](../../sql-reference/statements/create/view.md#materialized).
- `'Live' = 3` — [live представления](../../sql-reference/statements/create/view.md#live-view).
- `view_query` ([String](../../sql-reference/data-types/string.md)) — запрос, выполняемый представлением.
- `view_target` ([String](../../sql-reference/data-types/string.md)) — имя целевой таблицы представления.
- `read_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — количество прочитанных строк.
- `read_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — количество прочитанных байт.
- `written_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — количество записанных строк.
- `written_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — количество записанных байт.
- `peak_memory_usage` ([Int64](../../sql-reference/data-types/int-uint.md)) — максимальная разница между объемом выделенной и освобожденной памяти в контексте этого представления.
- `ProfileEvents` ([Map(String, UInt64)](../../sql-reference/data-types/array.md)) — события профиля, которые измеряют различные показатели. Их описание можно найти в таблице [system.events](../../operations/system-tables/events.md#system_tables-events).
- `status` ([Enum8](../../sql-reference/data-types/enum.md)) — статус представления. Возможные значения:
- `'QueryStart' = 1` — успешное начало выполнения представления. Не должно отображаться.
- `'QueryFinish' = 2` — успешное завершение выполнения представления.
- `'ExceptionBeforeStart' = 3` — исключение до начала выполнения представления.
- `'ExceptionWhileProcessing' = 4` — исключение во время выполнения представления.
- `exception_code` ([Int32](../../sql-reference/data-types/int-uint.md)) — код исключения.
- `exception` ([String](../../sql-reference/data-types/string.md)) — сообщение исключения.
- `stack_trace` ([String](../../sql-reference/data-types/string.md)) — [трассировка стека](https://ru.wikipedia.org/wiki/Трассировка_стека). Пустая строка, если запрос был успешно выполнен.
**Пример**
Запрос:
``` sql
SELECT * FROM system.query_views_log LIMIT 1 \G;
```
Результат:
``` text
Row 1:
──────
event_date: 2021-06-22
event_time: 2021-06-22 13:23:07
event_time_microseconds: 2021-06-22 13:23:07.738221
view_duration_ms: 0
initial_query_id: c3a1ac02-9cad-479b-af54-9e9c0a7afd70
view_name: default.matview_inner
view_uuid: 00000000-0000-0000-0000-000000000000
view_type: Materialized
view_query: SELECT * FROM default.table_b
view_target: default.`.inner.matview_inner`
read_rows: 4
read_bytes: 64
written_rows: 2
written_bytes: 32
peak_memory_usage: 4196188
ProfileEvents: {'FileOpen':2,'WriteBufferFromFileDescriptorWrite':2,'WriteBufferFromFileDescriptorWriteBytes':187,'IOBufferAllocs':3,'IOBufferAllocBytes':3145773,'FunctionExecute':3,'DiskWriteElapsedMicroseconds':13,'InsertedRows':2,'InsertedBytes':16,'SelectedRows':4,'SelectedBytes':48,'ContextLock':16,'RWLockAcquiredReadLocks':1,'RealTimeMicroseconds':698,'SoftPageFaults':4,'OSReadChars':463}
status: QueryFinish
exception_code: 0
exception:
stack_trace:
```
**См. также**
- [system.query_log](../../operations/system-tables/query_log.md#system_tables-query_log) — описание системной таблицы `query_log`, которая содержит общую информацию о выполненных запросах.
- [system.query_thread_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log) — описание системной таблицы `query_thread_log`, которая содержит информацию о каждом потоке выполнения запроса.

View File

@ -357,7 +357,7 @@ Result:
## multiFuzzyMatchAny(haystack, distance, \[pattern<sub>1</sub>, pattern<sub>2</sub>, …, pattern<sub>n</sub>\]) {#multifuzzymatchanyhaystack-distance-pattern1-pattern2-patternn}
То же, что и `multiMatchAny`, но возвращает 1 если любой pattern соответствует haystack в пределах константного [редакционного расстояния](https://en.wikipedia.org/wiki/Edit_distance). Эта функция также находится в экспериментальном режиме и может быть очень медленной. За подробностями обращайтесь к [документации hyperscan](https://intel.github.io/hyperscan/dev-reference/compilation.html#approximate-matching).
То же, что и `multiMatchAny`, но возвращает 1 если любой шаблон соответствует haystack в пределах константного [редакционного расстояния](https://en.wikipedia.org/wiki/Edit_distance). Эта функция основана на экспериментальной библиотеке [hyperscan](https://intel.github.io/hyperscan/dev-reference/compilation.html#approximate-matching) и может быть медленной для некоторых частных случаев. Производительность зависит от значения редакционного расстояния и используемых шаблонов, но всегда медленнее по сравнению с non-fuzzy вариантами.
## multiFuzzyMatchAnyIndex(haystack, distance, \[pattern<sub>1</sub>, pattern<sub>2</sub>, …, pattern<sub>n</sub>\]) {#multifuzzymatchanyindexhaystack-distance-pattern1-pattern2-patternn}

View File

@ -473,3 +473,7 @@ if (ENABLE_TESTS AND USE_GTEST)
add_custom_target (clickhouse-tests ALL DEPENDS ${CLICKHOUSE_UNIT_TESTS_TARGETS})
add_dependencies(clickhouse-bundle clickhouse-tests)
endif()
if (ENABLE_FUZZING)
add_compile_definitions(FUZZING_MODE=1)
endif ()

View File

@ -17,3 +17,9 @@ clickhouse_program_add(local)
if(NOT CLICKHOUSE_ONE_SHARED)
target_link_libraries(clickhouse-local-lib PRIVATE clickhouse-server-lib)
endif()
if (ENABLE_FUZZING)
add_compile_definitions(FUZZING_MODE=1)
set (WITH_COVERAGE ON)
target_link_libraries(clickhouse-local-lib PRIVATE ${LIB_FUZZING_ENGINE})
endif ()

View File

@ -41,6 +41,10 @@
#include <base/argsToConfig.h>
#include <filesystem>
#if defined(FUZZING_MODE)
#include <Functions/getFuzzerData.h>
#endif
namespace fs = std::filesystem;
@ -407,10 +411,25 @@ try
std::cout << std::fixed << std::setprecision(3);
std::cerr << std::fixed << std::setprecision(3);
#if defined(FUZZING_MODE)
static bool first_time = true;
if (first_time)
{
if (queries_files.empty() && !config().has("query"))
{
std::cerr << "\033[31m" << "ClickHouse compiled in fuzzing mode." << "\033[0m" << std::endl;
std::cerr << "\033[31m" << "You have to provide a query with --query or --queries-file option." << "\033[0m" << std::endl;
std::cerr << "\033[31m" << "The query have to use function getFuzzerData() inside." << "\033[0m" << std::endl;
exit(1);
}
is_interactive = false;
#else
is_interactive = stdin_is_a_tty
&& (config().hasOption("interactive")
|| (!config().has("query") && !config().has("table-structure") && queries_files.empty()));
#endif
if (!is_interactive)
{
/// We will terminate process on error
@ -439,6 +458,11 @@ try
connect();
#ifdef FUZZING_MODE
first_time = false;
}
#endif
if (is_interactive && !delayed_interactive)
{
runInteractive();
@ -451,7 +475,9 @@ try
runInteractive();
}
#ifndef FUZZING_MODE
cleanup();
#endif
return Application::EXIT_OK;
}
catch (const DB::Exception & e)
@ -653,7 +679,7 @@ void LocalServer::processConfig()
}
static std::string getHelpHeader()
[[ maybe_unused ]] static std::string getHelpHeader()
{
return
"usage: clickhouse-local [initial table definition] [--query <query>]\n"
@ -669,7 +695,7 @@ static std::string getHelpHeader()
}
static std::string getHelpFooter()
[[ maybe_unused ]] static std::string getHelpFooter()
{
return
"Example printing memory used by each Unix user:\n"
@ -680,11 +706,23 @@ static std::string getHelpFooter()
}
void LocalServer::printHelpMessage(const OptionsDescription & options_description)
void LocalServer::printHelpMessage([[maybe_unused]] const OptionsDescription & options_description)
{
#if defined(FUZZING_MODE)
std::cout <<
"usage: clickhouse <clickhouse-local arguments> -- <libfuzzer arguments>\n"
"Note: It is important not to use only one letter keys with single dash for \n"
"for clickhouse-local arguments. It may work incorrectly.\n"
"ClickHouse is build with coverage guided fuzzer (libfuzzer) inside it.\n"
"You have to provide a query which contains getFuzzerData function.\n"
"This will take the data from fuzzing engine, pass it to getFuzzerData function and execute a query.\n"
"Each time the data will be different, and it will last until some segfault or sanitizer assertion is found. \n";
#else
std::cout << getHelpHeader() << "\n";
std::cout << options_description.main_description.value() << "\n";
std::cout << getHelpFooter() << "\n";
#endif
}
@ -781,3 +819,51 @@ int mainEntryClickHouseLocal(int argc, char ** argv)
return code ? code : 1;
}
}
#if defined(FUZZING_MODE)
std::optional<DB::LocalServer> fuzz_app;
extern "C" int LLVMFuzzerInitialize(int * pargc, char *** pargv)
{
int & argc = *pargc;
char ** argv = *pargv;
/// As a user you can add flags to clickhouse binary in fuzzing mode as follows
/// clickhouse <set of clickhouse-local specific flag> -- <set of libfuzzer flags>
/// Calculate the position of delimiter "--" that separates arguments
/// of clickhouse-local and libfuzzer
int pos_delim = argc;
for (int i = 0; i < argc; ++i)
{
if (strcmp(argv[i], "--") == 0)
{
pos_delim = i;
break;
}
}
/// Initialize clickhouse-local app
fuzz_app.emplace();
fuzz_app->init(pos_delim, argv);
/// We will leave clickhouse-local specific arguments as is, because libfuzzer will ignore
/// all keys starting with --
return 0;
}
extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size)
try
{
auto input = String(reinterpret_cast<const char *>(data), size);
DB::FunctionGetFuzzerData::update(input);
fuzz_app->run();
return 0;
}
catch (...)
{
return 1;
}
#endif

View File

@ -88,6 +88,7 @@ namespace
using MainFunc = int (*)(int, char**);
#if !defined(FUZZING_MODE)
/// Add an item here to register new application
std::pair<const char *, MainFunc> clickhouse_applications[] =
@ -141,7 +142,6 @@ std::pair<const char *, MainFunc> clickhouse_applications[] =
{"hash-binary", mainEntryClickHouseHashBinary},
};
int printHelp(int, char **)
{
std::cerr << "Use one of the following commands:" << std::endl;
@ -150,7 +150,6 @@ int printHelp(int, char **)
return -1;
}
bool isClickhouseApp(const std::string & app_suffix, std::vector<char *> & argv)
{
/// Use app if the first arg 'app' is passed (the arg should be quietly removed)
@ -170,6 +169,7 @@ bool isClickhouseApp(const std::string & app_suffix, std::vector<char *> & argv)
std::string app_name = "clickhouse-" + app_suffix;
return !argv.empty() && (app_name == argv[0] || endsWith(argv[0], "/" + app_name));
}
#endif
enum class InstructionFail
@ -342,9 +342,13 @@ struct Checker
///
/// extern bool inside_main;
/// class C { C() { assert(inside_main); } };
#ifndef FUZZING_MODE
bool inside_main = false;
#else
bool inside_main = true;
#endif
#if !defined(FUZZING_MODE)
int main(int argc_, char ** argv_)
{
inside_main = true;
@ -375,3 +379,4 @@ int main(int argc_, char ** argv_)
return main_func(static_cast<int>(argv.size()), argv.data());
}
#endif

View File

@ -20,6 +20,7 @@
namespace DB
{
struct Settings;
template <typename T> constexpr bool DecimalOrExtendedInt =
@ -42,39 +43,19 @@ struct AvgFraction
/// Invoked only is either Numerator or Denominator are Decimal.
Float64 NO_SANITIZE_UNDEFINED divideIfAnyDecimal(UInt32 num_scale, UInt32 denom_scale [[maybe_unused]]) const
{
if constexpr (is_decimal<Numerator> && is_decimal<Denominator>)
{
// According to the docs, num(S1) / denom(S2) would have scale S1
if constexpr (std::is_same_v<Numerator, Decimal256> && std::is_same_v<Denominator, Decimal128>)
///Special case as Decimal256 / Decimal128 = compile error (as Decimal128 is not parametrized by a wide
///int), but an __int128 instead
return DecimalUtils::convertTo<Float64>(
numerator / (denominator.template convertTo<Decimal256>()), num_scale);
else
return DecimalUtils::convertTo<Float64>(numerator / denominator, num_scale);
}
/// Numerator is always casted to Float64 to divide correctly if the denominator is not Float64.
Float64 num_converted;
Float64 numerator_float;
if constexpr (is_decimal<Numerator>)
num_converted = DecimalUtils::convertTo<Float64>(numerator, num_scale);
numerator_float = DecimalUtils::convertTo<Float64>(numerator, num_scale);
else
num_converted = static_cast<Float64>(numerator); /// all other types, including extended integral.
std::conditional_t<DecimalOrExtendedInt<Denominator>,
Float64, Denominator> denom_converted;
numerator_float = numerator;
Float64 denominator_float;
if constexpr (is_decimal<Denominator>)
denom_converted = DecimalUtils::convertTo<Float64>(denominator, denom_scale);
else if constexpr (DecimalOrExtendedInt<Denominator>)
/// no way to divide Float64 and extended integral type without an explicit cast.
denom_converted = static_cast<Float64>(denominator);
denominator_float = DecimalUtils::convertTo<Float64>(denominator, denom_scale);
else
denom_converted = denominator; /// can divide on float, no cast required.
denominator_float = denominator;
return num_converted / denom_converted;
return numerator_float / denominator_float;
}
Float64 NO_SANITIZE_UNDEFINED divide() const

View File

@ -82,17 +82,17 @@ createAggregateFunctionAvgWeighted(const std::string & name, const DataTypes & a
const bool left_decimal = isDecimal(data_type);
const bool right_decimal = isDecimal(data_type_weight);
/// We multiply value by weight, so actual scale of numerator is <scale of value> + <scale of weight>
if (left_decimal && right_decimal)
ptr.reset(create(*data_type, *data_type_weight,
argument_types,
getDecimalScale(*data_type), getDecimalScale(*data_type_weight)));
getDecimalScale(*data_type) + getDecimalScale(*data_type_weight), getDecimalScale(*data_type_weight)));
else if (left_decimal)
ptr.reset(create(*data_type, *data_type_weight, argument_types,
getDecimalScale(*data_type)));
else if (right_decimal)
ptr.reset(create(*data_type, *data_type_weight, argument_types,
// numerator is not decimal, so its scale is 0
0, getDecimalScale(*data_type_weight)));
getDecimalScale(*data_type_weight), getDecimalScale(*data_type_weight)));
else
ptr.reset(create(*data_type, *data_type_weight, argument_types));

View File

@ -547,6 +547,7 @@ if (ENABLE_TESTS AND USE_GTEST)
clickhouse_parsers
clickhouse_storages_system
dbms
clickhouse_common_config
clickhouse_common_zookeeper
string_utils)

View File

@ -0,0 +1,7 @@
#include "gtest_global_context.h"
const ContextHolder & getContext()
{
static ContextHolder holder;
return holder;
}

View File

@ -18,8 +18,4 @@ struct ContextHolder
ContextHolder(ContextHolder &&) = default;
};
inline const ContextHolder & getContext()
{
static ContextHolder holder;
return holder;
}
const ContextHolder & getContext();

View File

@ -123,3 +123,7 @@ set_source_files_properties("pointInPolygon.cpp" PROPERTIES COMPILE_FLAGS -fno-s
# target_link_libraries(clickhouse_functions PRIVATE ${S2_LIBRARY})
target_include_directories(clickhouse_functions SYSTEM PUBLIC ${S2_GEOMETRY_INCLUDE_DIR})
if (ENABLE_FUZZING)
add_compile_definitions(FUZZING_MODE=1)
endif ()

View File

@ -0,0 +1,11 @@
#include <Functions/getFuzzerData.h>
namespace DB
{
void registerFunctionGetFuzzerData(FunctionFactory & factory)
{
factory.registerFunction<FunctionGetFuzzerData>();
}
}

View File

@ -0,0 +1,50 @@
#pragma once
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <DataTypes/DataTypeString.h>
#include <Core/Field.h>
namespace DB
{
class FunctionGetFuzzerData : public IFunction
{
inline static String fuzz_data;
public:
static constexpr auto name = "getFuzzerData";
inline static FunctionPtr create(ContextPtr) { return create(); }
static FunctionPtr create()
{
return std::make_shared<FunctionGetFuzzerData>();
}
inline String getName() const override { return name; }
inline size_t getNumberOfArguments() const override { return 0; }
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override
{
return std::make_shared<DataTypeString>();
}
inline bool isDeterministic() const override { return false; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName &,
const DataTypePtr &,
size_t input_rows_count) const override
{
return DataTypeString().createColumnConst(input_rows_count, fuzz_data);
}
static void update(const String & fuzz_data_)
{
fuzz_data = fuzz_data_;
}
};
}

View File

@ -85,6 +85,10 @@ void registerFunctionGetOSKernelVersion(FunctionFactory &);
void registerFunctionConvertCharset(FunctionFactory &);
#endif
#ifdef FUZZING_MODE
void registerFunctionGetFuzzerData(FunctionFactory & factory);
#endif
void registerFunctionsMiscellaneous(FunctionFactory & factory)
{
registerFunctionCurrentDatabase(factory);
@ -166,6 +170,10 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
#if USE_ICU
registerFunctionConvertCharset(factory);
#endif
#ifdef FUZZING_MODE
registerFunctionGetFuzzerData(factory);
#endif
}
}

View File

@ -465,9 +465,13 @@ void removeUnneededColumnsFromSelectClause(const ASTSelectQuery * select_query,
ASTFunction * func = elem->as<ASTFunction>();
/// Never remove untuple. It's result column may be in required columns.
/// It is not easy to analyze untuple here, because types were not calculated yes.
/// It is not easy to analyze untuple here, because types were not calculated yet.
if (func && func->name == "untuple")
new_elements.push_back(elem);
/// removing aggregation can change number of rows, so `count()` result in outer sub-query would be wrong
if (func && AggregateFunctionFactory::instance().isAggregateFunctionName(func->name) && !select_query->groupBy())
new_elements.push_back(elem);
}
}

View File

@ -0,0 +1,493 @@
#include <base/find_symbols.h>
#include <Interpreters/Context.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
#include <DataTypes/DataTypesNumber.h>
#include <Processors/Merges/Algorithms/Graphite.h>
#include <string_view>
#include <vector>
#include <unordered_map>
#include <fmt/format.h>
#include <Poco/Util/AbstractConfiguration.h>
using namespace std::literals;
namespace DB::ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int BAD_ARGUMENTS;
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
extern const int NO_ELEMENTS_IN_CONFIG;
}
namespace DB::Graphite
{
static std::unordered_map<RuleType, const String> ruleTypeMap =
{
{ RuleTypeAll, "all" },
{ RuleTypePlain, "plain" },
{ RuleTypeTagged, "tagged"},
{ RuleTypeTagList, "tag_list"}
};
const String & ruleTypeStr(RuleType rule_type)
{
try
{
return ruleTypeMap.at(rule_type);
}
catch (...)
{
throw Exception("invalid rule type: " + std::to_string(rule_type), DB::ErrorCodes::BAD_ARGUMENTS);
}
}
RuleType ruleType(const String & s)
{
if (s == "all")
return RuleTypeAll;
else if (s == "plain")
return RuleTypePlain;
else if (s == "tagged")
return RuleTypeTagged;
else if (s == "tag_list")
return RuleTypeTagList;
else
throw Exception("invalid rule type: " + s, DB::ErrorCodes::BAD_ARGUMENTS);
}
static const Graphite::Pattern undef_pattern =
{ /// empty pattern for selectPatternForPath
.rule_type = RuleTypeAll,
.regexp = nullptr,
.regexp_str = "",
.function = nullptr,
.retentions = Graphite::Retentions(),
.type = undef_pattern.TypeUndef,
};
inline static const Patterns & selectPatternsForMetricType(const Graphite::Params & params, const StringRef path)
{
if (params.patterns_typed)
{
std::string_view path_view = path.toView();
if (path_view.find("?"sv) == path_view.npos)
return params.patterns_plain;
else
return params.patterns_tagged;
}
else
{
return params.patterns;
}
}
Graphite::RollupRule selectPatternForPath(
const Graphite::Params & params,
const StringRef path)
{
const Graphite::Pattern * first_match = &undef_pattern;
const Patterns & patterns_check = selectPatternsForMetricType(params, path);
for (const auto & pattern : patterns_check)
{
if (!pattern.regexp)
{
/// Default pattern
if (first_match->type == first_match->TypeUndef && pattern.type == pattern.TypeAll)
{
/// There is only default pattern for both retention and aggregation
return std::pair(&pattern, &pattern);
}
if (pattern.type != first_match->type)
{
if (first_match->type == first_match->TypeRetention)
{
return std::pair(first_match, &pattern);
}
if (first_match->type == first_match->TypeAggregation)
{
return std::pair(&pattern, first_match);
}
}
}
else
{
if (pattern.regexp->match(path.data, path.size))
{
/// General pattern with matched path
if (pattern.type == pattern.TypeAll)
{
/// Only for not default patterns with both function and retention parameters
return std::pair(&pattern, &pattern);
}
if (first_match->type == first_match->TypeUndef)
{
first_match = &pattern;
continue;
}
if (pattern.type != first_match->type)
{
if (first_match->type == first_match->TypeRetention)
{
return std::pair(first_match, &pattern);
}
if (first_match->type == first_match->TypeAggregation)
{
return std::pair(&pattern, first_match);
}
}
}
}
}
return {nullptr, nullptr};
}
/** Is used to order Graphite::Retentions by age and precision descending.
* Throws exception if not both age and precision are less or greater then another.
*/
static bool compareRetentions(const Retention & a, const Retention & b)
{
if (a.age > b.age && a.precision > b.precision)
{
return true;
}
else if (a.age < b.age && a.precision < b.precision)
{
return false;
}
String error_msg = "age and precision should only grow up: "
+ std::to_string(a.age) + ":" + std::to_string(a.precision) + " vs "
+ std::to_string(b.age) + ":" + std::to_string(b.precision);
throw Exception(
error_msg,
DB::ErrorCodes::BAD_ARGUMENTS);
}
bool operator==(const Retention & a, const Retention & b)
{
return a.age == b.age && a.precision == b.precision;
}
std::ostream & operator<<(std::ostream & stream, const Retentions & a)
{
stream << "{ ";
for (size_t i = 0; i < a.size(); i++)
{
if (i > 0)
stream << ",";
stream << " { age = " << a[i].age << ", precision = " << a[i].precision << " }";
}
stream << " }";
return stream;
}
bool operator==(const Pattern & a, const Pattern & b)
{
// equal
// Retentions retentions; /// Must be ordered by 'age' descending.
if (a.type != b.type || a.regexp_str != b.regexp_str || a.rule_type != b.rule_type)
return false;
if (a.function == nullptr)
{
if (b.function != nullptr)
return false;
}
else if (b.function == nullptr)
{
return false;
}
else if (a.function->getName() != b.function->getName())
{
return false;
}
return a.retentions == b.retentions;
}
std::ostream & operator<<(std::ostream & stream, const Pattern & a)
{
stream << "{ rule_type = " << ruleTypeStr(a.rule_type);
if (!a.regexp_str.empty())
stream << ", regexp = '" << a.regexp_str << "'";
if (a.function != nullptr)
stream << ", function = " << a.function->getName();
if (!a.retentions.empty())
{
stream << ",\n retentions = {\n";
for (size_t i = 0; i < a.retentions.size(); i++)
{
stream << " { " << a.retentions[i].age << ", " << a.retentions[i].precision << " }";
if (i < a.retentions.size() - 1)
stream << ",";
stream << "\n";
}
stream << " }\n";
}
else
stream << " ";
stream << "}";
return stream;
}
std::string buildTaggedRegex(std::string regexp_str)
{
/*
* tags list in format (for name or any value can use regexp, alphabet sorting not needed)
* spaces are not stiped and used as tag and value part
* name must be first (if used)
*
* tag1=value1; tag2=VALUE2_REGEX;tag3=value3
* or
* name;tag1=value1;tag2=VALUE2_REGEX;tag3=value3
* or for one tag
* tag1=value1
*
* Resulting regex against metric like
* name?tag1=value1&tag2=value2
*
* So,
*
* name
* produce
* name\?
*
* tag2=val2
* produce
* [\?&]tag2=val2(&.*)?$
*
* nam.* ; tag1=val1 ; tag2=val2
* produce
* nam.*\?(.*&)?tag1=val1&(.*&)?tag2=val2(&.*)?$
*/
std::vector<std::string> tags;
splitInto<';'>(tags, regexp_str);
/* remove empthy elements */
using namespace std::string_literals;
tags.erase(std::remove(tags.begin(), tags.end(), ""s), tags.end());
if (tags[0].find('=') == tags[0].npos)
{
if (tags.size() == 1) /* only name */
return "^" + tags[0] + "\\?";
/* start with name value */
regexp_str = "^" + tags[0] + "\\?(.*&)?";
tags.erase(std::begin(tags));
}
else
regexp_str = "[\\?&]";
std::sort(std::begin(tags), std::end(tags)); /* sorted tag keys */
regexp_str += fmt::format(
"{}{}",
fmt::join(tags, "&(.*&)?"),
"(&.*)?$" /* close regex */
);
return regexp_str;
}
/** Read the settings for Graphite rollup from config.
* Example
*
* <graphite_rollup>
* <path_column_name>Path</path_column_name>
* <pattern>
* <regexp>click_cost</regexp>
* <function>any</function>
* <retention>
* <age>0</age>
* <precision>3600</precision>
* </retention>
* <retention>
* <age>86400</age>
* <precision>60</precision>
* </retention>
* </pattern>
* <default>
* <function>max</function>
* <retention>
* <age>0</age>
* <precision>60</precision>
* </retention>
* <retention>
* <age>3600</age>
* <precision>300</precision>
* </retention>
* <retention>
* <age>86400</age>
* <precision>3600</precision>
* </retention>
* </default>
* </graphite_rollup>
*/
static const Pattern &
appendGraphitePattern(
const Poco::Util::AbstractConfiguration & config,
const String & config_element, Patterns & patterns,
bool default_rule,
ContextPtr context)
{
Pattern pattern;
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_element, keys);
for (const auto & key : keys)
{
if (key == "regexp")
{
pattern.regexp_str = config.getString(config_element + ".regexp");
}
else if (key == "function")
{
String aggregate_function_name_with_params = config.getString(config_element + ".function");
String aggregate_function_name;
Array params_row;
getAggregateFunctionNameAndParametersArray(
aggregate_function_name_with_params, aggregate_function_name, params_row, "GraphiteMergeTree storage initialization", context);
/// TODO Not only Float64
AggregateFunctionProperties properties;
pattern.function = AggregateFunctionFactory::instance().get(
aggregate_function_name, {std::make_shared<DataTypeFloat64>()}, params_row, properties);
}
else if (key == "rule_type")
{
String rule_type = config.getString(config_element + ".rule_type");
pattern.rule_type = ruleType(rule_type);
}
else if (startsWith(key, "retention"))
{
pattern.retentions.emplace_back(Graphite::Retention{
.age = config.getUInt(config_element + "." + key + ".age"),
.precision = config.getUInt(config_element + "." + key + ".precision")});
}
else
throw Exception("Unknown element in config: " + key, DB::ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
}
if (!pattern.regexp_str.empty())
{
if (pattern.rule_type == RuleTypeTagList)
{
// construct tagged regexp
pattern.regexp_str = buildTaggedRegex(pattern.regexp_str);
pattern.rule_type = RuleTypeTagged;
}
pattern.regexp = std::make_shared<OptimizedRegularExpression>(pattern.regexp_str);
}
if (!pattern.function && pattern.retentions.empty())
throw Exception(
"At least one of an aggregate function or retention rules is mandatory for rollup patterns in GraphiteMergeTree",
DB::ErrorCodes::NO_ELEMENTS_IN_CONFIG);
if (default_rule && pattern.rule_type != RuleTypeAll)
{
throw Exception(
"Default must have rule_type all for rollup patterns in GraphiteMergeTree",
DB::ErrorCodes::BAD_ARGUMENTS);
}
if (!pattern.function)
{
pattern.type = pattern.TypeRetention;
}
else if (pattern.retentions.empty())
{
pattern.type = pattern.TypeAggregation;
}
else
{
pattern.type = pattern.TypeAll;
}
if (pattern.type & pattern.TypeAggregation) /// TypeAggregation or TypeAll
if (pattern.function->allocatesMemoryInArena())
throw Exception(
"Aggregate function " + pattern.function->getName() + " isn't supported in GraphiteMergeTree", DB::ErrorCodes::NOT_IMPLEMENTED);
/// retention should be in descending order of age.
if (pattern.type & pattern.TypeRetention) /// TypeRetention or TypeAll
std::sort(pattern.retentions.begin(), pattern.retentions.end(), compareRetentions);
patterns.emplace_back(pattern);
return patterns.back();
}
void setGraphitePatternsFromConfig(ContextPtr context, const String & config_element, Graphite::Params & params)
{
const auto & config = context->getConfigRef();
if (!config.has(config_element))
throw Exception("No '" + config_element + "' element in configuration file", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
params.config_name = config_element;
params.path_column_name = config.getString(config_element + ".path_column_name", "Path");
params.time_column_name = config.getString(config_element + ".time_column_name", "Time");
params.value_column_name = config.getString(config_element + ".value_column_name", "Value");
params.version_column_name = config.getString(config_element + ".version_column_name", "Timestamp");
params.patterns_typed = false;
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_element, keys);
for (const auto & key : keys)
{
if (startsWith(key, "pattern"))
{
if (appendGraphitePattern(config, config_element + "." + key, params.patterns, false, context).rule_type != RuleTypeAll)
params.patterns_typed = true;
}
else if (key == "default")
{
/// See below.
}
else if (key == "path_column_name" || key == "time_column_name" || key == "value_column_name" || key == "version_column_name")
{
/// See above.
}
else
throw Exception("Unknown element in config: " + key, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
}
if (config.has(config_element + ".default"))
appendGraphitePattern(config, config_element + "." + ".default", params.patterns, true, context);
for (const auto & pattern : params.patterns)
{
if (pattern.rule_type == RuleTypeAll)
{
if (params.patterns_typed)
{
params.patterns_plain.push_back(pattern);
params.patterns_tagged.push_back(pattern);
}
}
else if (pattern.rule_type == RuleTypePlain)
{
params.patterns_plain.push_back(pattern);
}
else if (pattern.rule_type == RuleTypeTagged)
{
params.patterns_tagged.push_back(pattern);
}
else
{
throw Exception("Unhandled rule_type in config: " + ruleTypeStr(pattern.rule_type), ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
}
}
}
}

View File

@ -1,13 +1,8 @@
#pragma once
#include <base/StringRef.h>
#include <Common/OptimizedRegularExpression.h>
namespace DB
{
class IAggregateFunction;
using AggregateFunctionPtr = std::shared_ptr<const IAggregateFunction>;
}
#include <AggregateFunctions/IAggregateFunction.h>
/** Intended for implementation of "rollup" - aggregation (rounding) of older data
* for a table with Graphite data (Graphite is the system for time series monitoring).
@ -97,16 +92,32 @@ using AggregateFunctionPtr = std::shared_ptr<const IAggregateFunction>;
namespace DB::Graphite
{
// sync with rule_types_str
enum RuleType
{
RuleTypeAll = 0, // default, with regex, compatible with old scheme
RuleTypePlain = 1, // plain metrics, with regex, compatible with old scheme
RuleTypeTagged = 2, // tagged metrics, with regex, compatible with old scheme
RuleTypeTagList = 3 // tagged metrics, with regex (converted to RuleTypeTagged from string like 'retention=10min ; env=(staging|prod)')
};
const String & ruleTypeStr(RuleType rule_type);
struct Retention
{
UInt32 age;
UInt32 precision;
};
bool operator==(const Retention & a, const Retention & b);
using Retentions = std::vector<Retention>;
std::ostream &operator<<(std::ostream & stream, const Retentions & a);
struct Pattern
{
RuleType rule_type = RuleTypeAll;
std::shared_ptr<OptimizedRegularExpression> regexp;
std::string regexp_str;
AggregateFunctionPtr function;
@ -114,6 +125,9 @@ struct Pattern
enum { TypeUndef, TypeRetention, TypeAggregation, TypeAll } type = TypeAll; /// The type of defined pattern, filled automatically
};
bool operator==(const Pattern & a, const Pattern & b);
std::ostream &operator<<(std::ostream & stream, const Pattern & a);
using Patterns = std::vector<Pattern>;
using RetentionPattern = Pattern;
using AggregationPattern = Pattern;
@ -125,9 +139,16 @@ struct Params
String time_column_name;
String value_column_name;
String version_column_name;
bool patterns_typed;
Graphite::Patterns patterns;
Graphite::Patterns patterns_plain;
Graphite::Patterns patterns_tagged;
};
using RollupRule = std::pair<const RetentionPattern *, const AggregationPattern *>;
Graphite::RollupRule selectPatternForPath(const Graphite::Params & params, const StringRef path);
void setGraphitePatternsFromConfig(ContextPtr context, const String & config_element, Graphite::Params & params);
}

View File

@ -1,3 +1,4 @@
#include <Processors/Merges/Algorithms/Graphite.h>
#include <Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <base/DateLUTImpl.h>
@ -52,62 +53,6 @@ GraphiteRollupSortedAlgorithm::GraphiteRollupSortedAlgorithm(
columns_definition = defineColumns(header, params);
}
Graphite::RollupRule GraphiteRollupSortedAlgorithm::selectPatternForPath(StringRef path) const
{
const Graphite::Pattern * first_match = &undef_pattern;
for (const auto & pattern : params.patterns)
{
if (!pattern.regexp)
{
/// Default pattern
if (first_match->type == first_match->TypeUndef && pattern.type == pattern.TypeAll)
{
/// There is only default pattern for both retention and aggregation
return std::pair(&pattern, &pattern);
}
if (pattern.type != first_match->type)
{
if (first_match->type == first_match->TypeRetention)
{
return std::pair(first_match, &pattern);
}
if (first_match->type == first_match->TypeAggregation)
{
return std::pair(&pattern, first_match);
}
}
}
else if (pattern.regexp->match(path.data, path.size))
{
/// General pattern with matched path
if (pattern.type == pattern.TypeAll)
{
/// Only for not default patterns with both function and retention parameters
return std::pair(&pattern, &pattern);
}
if (first_match->type == first_match->TypeUndef)
{
first_match = &pattern;
continue;
}
if (pattern.type != first_match->type)
{
if (first_match->type == first_match->TypeRetention)
{
return std::pair(first_match, &pattern);
}
if (first_match->type == first_match->TypeAggregation)
{
return std::pair(&pattern, first_match);
}
}
}
}
return {nullptr, nullptr};
}
UInt32 GraphiteRollupSortedAlgorithm::selectPrecision(const Graphite::Retentions & retentions, time_t time) const
{
static_assert(is_signed_v<time_t>, "time_t must be signed type");
@ -188,7 +133,7 @@ IMergingAlgorithm::Status GraphiteRollupSortedAlgorithm::merge()
Graphite::RollupRule next_rule = merged_data.currentRule();
if (new_path)
next_rule = selectPatternForPath(next_path);
next_rule = selectPatternForPath(this->params, next_path);
const Graphite::RetentionPattern * retention_pattern = std::get<0>(next_rule);
time_t next_time_rounded;

View File

@ -102,16 +102,6 @@ private:
time_t current_time = 0;
time_t current_time_rounded = 0;
const Graphite::Pattern undef_pattern =
{ /// temporary empty pattern for selectPatternForPath
.regexp = nullptr,
.regexp_str = "",
.function = nullptr,
.retentions = DB::Graphite::Retentions(),
.type = undef_pattern.TypeUndef,
};
Graphite::RollupRule selectPatternForPath(StringRef path) const;
UInt32 selectPrecision(const Graphite::Retentions & retentions, time_t time) const;
/// Insert the values into the resulting columns, which will not be changed in the future.

View File

@ -0,0 +1,597 @@
#include <cstring>
#include <filesystem>
#include <fstream>
#include <stdexcept>
#include <gtest/gtest.h>
#include <Common/tests/gtest_global_context.h>
#include <Common/tests/gtest_global_register.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <Processors/Merges/Algorithms/Graphite.h>
#include <Common/Config/ConfigProcessor.h>
using namespace DB;
static int regAggregateFunctions = 0;
void tryRegisterAggregateFunctions()
{
if (!regAggregateFunctions)
{
registerAggregateFunctions();
regAggregateFunctions = 1;
}
}
static ConfigProcessor::LoadedConfig loadConfiguration(const std::string & config_path)
{
ConfigProcessor config_processor(config_path, true, true);
ConfigProcessor::LoadedConfig config = config_processor.loadConfig(false);
return config;
}
static ConfigProcessor::LoadedConfig loadConfigurationFromString(std::string & s)
{
char tmp_file[19];
strcpy(tmp_file, "/tmp/rollup-XXXXXX");
int fd = mkstemp(tmp_file);
if (fd == -1)
{
throw std::runtime_error(strerror(errno));
}
try {
if (write(fd, s.c_str(), s.size()) < s.size())
{
throw std::runtime_error("unable write to temp file");
}
if (write(fd, "\n", 1) != 1)
{
throw std::runtime_error("unable write to temp file");
}
close(fd);
auto config_path = std::string(tmp_file) + ".xml";
if (std::rename(tmp_file, config_path.c_str()))
{
int err = errno;
remove(tmp_file);
throw std::runtime_error(strerror(err));
}
ConfigProcessor::LoadedConfig config = loadConfiguration(config_path);
remove(tmp_file);
return config;
}
catch (...)
{
remove(tmp_file);
throw;
}
}
static Graphite::Params setGraphitePatterns(ContextMutablePtr context, ConfigProcessor::LoadedConfig & config)
{
context->setConfig(config.configuration);
Graphite::Params params;
setGraphitePatternsFromConfig(context, "graphite_rollup", params);
return params;
}
struct PatternForCheck
{
Graphite::RuleType rule_type;
std::string regexp_str;
String function;
Graphite::Retentions retentions;
};
bool checkRule(const Graphite::Pattern & pattern, const struct PatternForCheck & pattern_check,
const std::string & typ, const std::string & path, std::string & message)
{
bool rule_type_eq = (pattern.rule_type == pattern_check.rule_type);
bool regexp_eq = (pattern.regexp_str == pattern_check.regexp_str);
bool function_eq = (pattern.function == nullptr && pattern_check.function.empty())
|| (pattern.function != nullptr && pattern.function->getName() == pattern_check.function);
bool retentions_eq = (pattern.retentions == pattern_check.retentions);
if (rule_type_eq && regexp_eq && function_eq && retentions_eq)
return true;
message = typ + " rollup rule mismatch for '" + path + "'," +
(rule_type_eq ? "" : "rule_type ") +
(regexp_eq ? "" : "regexp ") +
(function_eq ? "" : "function ") +
(retentions_eq ? "" : "retentions ");
return false;
}
std::ostream & operator<<(std::ostream & stream, const PatternForCheck & a)
{
stream << "{ rule_type = " << ruleTypeStr(a.rule_type);
if (!a.regexp_str.empty())
stream << ", regexp = '" << a.regexp_str << "'";
if (!a.function.empty())
stream << ", function = " << a.function;
if (!a.retentions.empty())
{
stream << ",\n retentions = {\n";
for (size_t i = 0; i < a.retentions.size(); i++)
{
stream << " { " << a.retentions[i].age << ", " << a.retentions[i].precision << " }";
if (i < a.retentions.size() - 1)
stream << ",";
stream << "\n";
}
stream << " }\n";
}
else
stream << " ";
stream << "}";
return stream;
}
struct PatternsForPath
{
std::string path;
PatternForCheck retention_want;
PatternForCheck aggregation_want;
};
TEST(GraphiteTest, testSelectPattern)
{
tryRegisterAggregateFunctions();
using namespace std::literals;
std::string
xml(R"END(<yandex>
<graphite_rollup>
<pattern>
<regexp>\.sum$</regexp>
<function>sum</function>
</pattern>
<pattern>
<regexp>^((.*)|.)sum\?</regexp>
<function>sum</function>
</pattern>
<pattern>
<regexp>\.max$</regexp>
<function>max</function>
</pattern>
<pattern>
<regexp>^((.*)|.)max\?</regexp>
<function>max</function>
</pattern>
<pattern>
<regexp>\.min$</regexp>
<function>min</function>
</pattern>
<pattern>
<regexp>^((.*)|.)min\?</regexp>
<function>min</function>
</pattern>
<pattern>
<regexp>\.(count|sum|sum_sq)$</regexp>
<function>sum</function>
</pattern>
<pattern>
<regexp>^((.*)|.)(count|sum|sum_sq)\?</regexp>
<function>sum</function>
</pattern>
<pattern>
<regexp>^retention\.</regexp>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<default>
<function>avg</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>3600</age>
<precision>300</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</default>
</graphite_rollup>
</yandex>
)END");
// Retentions must be ordered by 'age' descending.
std::vector<struct PatternsForPath> tests
{
{
"test.sum",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypeAll, R"END(\.sum$)END", "sum", { } }
},
{
"val.sum?env=test&tag=Fake3",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypeAll, R"END(^((.*)|.)sum\?)END", "sum", { } }
},
{
"test.max",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypeAll, R"END(\.max$)END", "max", { } },
},
{
"val.max?env=test&tag=Fake4",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypeAll, R"END(^((.*)|.)max\?)END", "max", { } },
},
{
"test.min",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypeAll, R"END(\.min$)END", "min", { } },
},
{
"val.min?env=test&tag=Fake5",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypeAll, R"END(^((.*)|.)min\?)END", "min", { } },
},
{
"retention.count",
{ Graphite::RuleTypeAll, R"END(^retention\.)END", "", { { 86400, 3600 }, { 0, 60 } } }, // ^retention
{ Graphite::RuleTypeAll, R"END(\.(count|sum|sum_sq)$)END", "sum", { } },
},
{
"val.retention.count?env=test&tag=Fake5",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypeAll, R"END(^((.*)|.)(count|sum|sum_sq)\?)END", "sum", { } },
},
{
"val.count?env=test&tag=Fake5",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypeAll, R"END(^((.*)|.)(count|sum|sum_sq)\?)END", "sum", { } },
},
{
"test.p95",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
},
{
"val.p95?env=test&tag=FakeNo",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
},
{
"default",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
},
{
"val.default?env=test&tag=FakeNo",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
}
};
auto config = loadConfigurationFromString(xml);
ContextMutablePtr context = getContext().context;
Graphite::Params params = setGraphitePatterns(context, config);
for (const auto & t : tests)
{
auto rule = DB::Graphite::selectPatternForPath(params, t.path);
std:: string message;
if (!checkRule(*rule.first, t.retention_want, "retention", t.path, message))
ADD_FAILURE() << message << ", got\n" << *rule.first << "\n, want\n" << t.retention_want << "\n";
if (!checkRule(*rule.second, t.aggregation_want, "aggregation", t.path, message))
ADD_FAILURE() << message << ", got\n" << *rule.second << "\n, want\n" << t.aggregation_want << "\n";
}
}
namespace DB::Graphite
{
std::string buildTaggedRegex(std::string regexp_str);
}
struct RegexCheck
{
std::string regex;
std::string regex_want;
std::string match;
std::string nomatch;
};
TEST(GraphiteTest, testBuildTaggedRegex)
{
std::vector<struct RegexCheck> tests
{
{
"cpu\\.loadavg;project=DB.*;env=st.*",
R"END(^cpu\.loadavg\?(.*&)?env=st.*&(.*&)?project=DB.*(&.*)?$)END",
R"END(cpu.loadavg?env=staging&project=DBAAS)END",
R"END(cpu.loadavg?env=staging&project=D)END"
},
{
R"END(project=DB.*;env=staging;)END",
R"END([\?&]env=staging&(.*&)?project=DB.*(&.*)?$)END",
R"END(cpu.loadavg?env=staging&project=DBPG)END",
R"END(cpu.loadavg?env=stagingN&project=DBAAS)END"
},
{
"env=staging;",
R"END([\?&]env=staging(&.*)?$)END",
R"END(cpu.loadavg?env=staging&project=DPG)END",
R"END(cpu.loadavg?env=stagingN)END"
},
{
" env = staging ;", // spaces are allowed,
R"END([\?&] env = staging (&.*)?$)END",
R"END(cpu.loadavg? env = staging &project=DPG)END",
R"END(cpu.loadavg?env=stagingN)END"
},
{
"name;",
R"END(^name\?)END",
R"END(name?env=staging&project=DPG)END",
R"END(nameN?env=stagingN)END",
},
{
"name",
R"END(^name\?)END",
R"END(name?env=staging&project=DPG)END",
R"END(nameN?env=stagingN)END",
}
};
for (const auto & t : tests)
{
auto s = DB::Graphite::buildTaggedRegex(t.regex);
EXPECT_EQ(t.regex_want, s) << "result for '" << t.regex_want << "' mismatch";
auto regexp = OptimizedRegularExpression(s);
EXPECT_TRUE(regexp.match(t.match.data(), t.match.size())) << t.match << " match for '" << s << "' failed";
EXPECT_FALSE(regexp.match(t.nomatch.data(), t.nomatch.size())) << t.nomatch << " ! match for '" << s << "' failed";
}
}
TEST(GraphiteTest, testSelectPatternTyped)
{
tryRegisterAggregateFunctions();
using namespace std::literals;
std::string
xml(R"END(<yandex>
<graphite_rollup>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.sum$</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>tagged</rule_type>
<regexp>^((.*)|.)sum\?</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.max$</regexp>
<function>max</function>
</pattern>
<pattern>
<rule_type>tagged</rule_type>
<regexp>^((.*)|.)max\?</regexp>
<function>max</function>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.min$</regexp>
<function>min</function>
</pattern>
<pattern>
<rule_type>tagged</rule_type>
<regexp>^((.*)|.)min\?</regexp>
<function>min</function>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.(count|sum|sum_sq)$</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>tagged</rule_type>
<regexp>^((.*)|.)(count|sum|sum_sq)\?</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>^retention\.</regexp>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<rule_type>tagged</rule_type>
<regexp><![CDATA[[\?&]retention=hour(&.*)?$]]></regexp>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<rule_type>tag_list</rule_type>
<regexp>retention=10min;env=staging</regexp>
<retention>
<age>0</age>
<precision>600</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<rule_type>tag_list</rule_type>
<regexp>retention=10min;env=[A-Za-z-]+rod[A-Za-z-]+</regexp>
<retention>
<age>0</age>
<precision>600</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<rule_type>tag_list</rule_type>
<regexp>cpu\.loadavg</regexp>
<retention>
<age>0</age>
<precision>600</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<default>
<function>avg</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>3600</age>
<precision>300</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</default>
</graphite_rollup>
</yandex>
)END");
// Retentions must be ordered by 'age' descending.
std::vector<PatternsForPath> tests
{
{
"test.sum",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypePlain, R"END(\.sum$)END", "sum", { } }
},
{
"val.sum?env=test&tag=Fake3",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypeTagged, R"END(^((.*)|.)sum\?)END", "sum", { } }
},
{
"test.max",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypePlain, R"END(\.max$)END", "max", { } },
},
{
"val.max?env=test&tag=Fake4",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypeTagged, R"END(^((.*)|.)max\?)END", "max", { } },
},
{
"test.min",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypePlain, R"END(\.min$)END", "min", { } },
},
{
"val.min?env=test&tag=Fake5",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypeTagged, R"END(^((.*)|.)min\?)END", "min", { } },
},
{
"retention.count",
{ Graphite::RuleTypePlain, R"END(^retention\.)END", "", { { 86400, 3600 }, { 0, 60 } } }, // ^retention
{ Graphite::RuleTypePlain, R"END(\.(count|sum|sum_sq)$)END", "sum", { } },
},
{
"val.count?env=test&retention=hour&tag=Fake5",
{ Graphite::RuleTypeTagged, R"END([\?&]retention=hour(&.*)?$)END", "", { { 86400, 3600 }, { 0, 60 } } }, // tagged retention=hour
{ Graphite::RuleTypeTagged, R"END(^((.*)|.)(count|sum|sum_sq)\?)END", "sum", { } },
},
{
"val.count?env=test&retention=hour",
{ Graphite::RuleTypeTagged, R"END([\?&]retention=hour(&.*)?$)END", "", { { 86400, 3600 }, { 0, 60 } } }, // tagged retention=hour
{ Graphite::RuleTypeTagged, R"END(^((.*)|.)(count|sum|sum_sq)\?)END", "sum", { } },
},
{
"val.count?env=staging&retention=10min",
{ Graphite::RuleTypeTagged, R"END([\?&]env=staging&(.*&)?retention=10min(&.*)?$)END", "", { { 86400, 3600 }, { 0, 600 } } }, // retention=10min ; env=staging
{ Graphite::RuleTypeTagged, R"END(^((.*)|.)(count|sum|sum_sq)\?)END", "sum", { } },
},
{
"val.count?env=production&retention=10min",
{ Graphite::RuleTypeTagged, R"END([\?&]env=[A-Za-z-]+rod[A-Za-z-]+&(.*&)?retention=10min(&.*)?$)END", "", { { 86400, 3600 }, { 0, 600 } } }, // retention=10min ; env=[A-Za-z-]+rod[A-Za-z-]+
{ Graphite::RuleTypeTagged, R"END(^((.*)|.)(count|sum|sum_sq)\?)END", "sum", { } },
},
{
"val.count?env=test&tag=Fake5",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypeTagged, R"END(^((.*)|.)(count|sum|sum_sq)\?)END", "sum", { } },
},
{
"cpu.loadavg?env=test&tag=FakeNo",
{ Graphite::RuleTypeTagged, R"END(^cpu\.loadavg\?)END", "", { { 86400, 3600 }, { 0, 600 } } }, // name=cpu\.loadavg
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } },
},
{
"test.p95",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
},
{
"val.p95?env=test&tag=FakeNo",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
},
{
"default",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
},
{
"val.default?env=test&tag=FakeNo",
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
{ Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default
}
};
auto config = loadConfigurationFromString(xml);
ContextMutablePtr context = getContext().context;
Graphite::Params params = setGraphitePatterns(context, config);
for (const auto & t : tests)
{
auto rule = DB::Graphite::selectPatternForPath(params, t.path);
std:: string message;
if (!checkRule(*rule.first, t.retention_want, "retention", t.path, message))
ADD_FAILURE() << message << ", got\n" << *rule.first << "\n, want\n" << t.retention_want << "\n";
if (!checkRule(*rule.second, t.aggregation_want, "aggregation", t.path, message))
ADD_FAILURE() << message << ", got\n" << *rule.second << "\n, want\n" << t.aggregation_want << "\n";
}
}

View File

@ -1,9 +1,6 @@
#pragma once
#include <functional>
#include <memory>
#include <base/logger_useful.h>
#include <Common/CurrentMetrics.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Core/BackgroundSchedulePool.h>
@ -17,135 +14,74 @@ namespace zkutil
*
* But then we decided to get rid of leader election, so every replica can become leader.
* For now, every replica can become leader if there is no leader among replicas with old version.
*
* It's tempting to remove this class at all, but we have to maintain it,
* to maintain compatibility when replicas with different versions work on the same cluster
* (this is allowed for short time period during cluster update).
*
* Replicas with new versions creates ephemeral sequential nodes with values like "replica_name (multiple leaders Ok)".
* If the first node belongs to a replica with new version, then all replicas with new versions become leaders.
*/
class LeaderElection
void checkNoOldLeaders(Poco::Logger * log, ZooKeeper & zookeeper, const String path)
{
public:
using LeadershipHandler = std::function<void()>;
/// Previous versions (before 21.12) used to create ephemeral sequential node path/leader_election-
/// Replica with the lexicographically smallest node name becomes leader (before 20.6) or enables multi-leader mode (since 20.6)
constexpr auto persistent_multiple_leaders = "leader_election-0"; /// Less than any sequential node
constexpr auto suffix = " (multiple leaders Ok)";
constexpr auto persistent_identifier = "all (multiple leaders Ok)";
/** handler is called when this instance become leader.
*
* identifier - if not empty, must uniquely (within same path) identify participant of leader election.
* It means that different participants of leader election have different identifiers
* and existence of more than one ephemeral node with same identifier indicates an error.
*/
LeaderElection(
DB::BackgroundSchedulePool & pool_,
const std::string & path_,
ZooKeeper & zookeeper_,
LeadershipHandler handler_,
const std::string & identifier_)
: pool(pool_), path(path_), zookeeper(zookeeper_), handler(handler_), identifier(identifier_ + suffix)
, log_name("LeaderElection (" + path + ")")
, log(&Poco::Logger::get(log_name))
size_t num_tries = 1000;
while (num_tries--)
{
task = pool.createTask(log_name, [this] { threadFunction(); });
createNode();
}
void shutdown()
{
if (shutdown_called)
Strings potential_leaders;
Coordination::Error code = zookeeper.tryGetChildren(path, potential_leaders);
/// NOTE zookeeper_path/leader_election node must exist now, but maybe we will remove it in future versions.
if (code == Coordination::Error::ZNONODE)
return;
else if (code != Coordination::Error::ZOK)
throw KeeperException(code, path);
shutdown_called = true;
task->deactivate();
}
Coordination::Requests ops;
~LeaderElection()
{
releaseNode();
}
private:
static inline constexpr auto suffix = " (multiple leaders Ok)";
DB::BackgroundSchedulePool & pool;
DB::BackgroundSchedulePool::TaskHolder task;
std::string path;
ZooKeeper & zookeeper;
LeadershipHandler handler;
std::string identifier;
std::string log_name;
Poco::Logger * log;
EphemeralNodeHolderPtr node;
std::string node_name;
std::atomic<bool> shutdown_called {false};
void createNode()
{
shutdown_called = false;
node = EphemeralNodeHolder::createSequential(fs::path(path) / "leader_election-", zookeeper, identifier);
std::string node_path = node->getPath();
node_name = node_path.substr(node_path.find_last_of('/') + 1);
task->activateAndSchedule();
}
void releaseNode()
{
shutdown();
node = nullptr;
}
void threadFunction()
{
bool success = false;
try
if (potential_leaders.empty())
{
Strings children = zookeeper.getChildren(path);
std::sort(children.begin(), children.end());
auto my_node_it = std::lower_bound(children.begin(), children.end(), node_name);
if (my_node_it == children.end() || *my_node_it != node_name)
throw Poco::Exception("Assertion failed in LeaderElection");
String value = zookeeper.get(path + "/" + children.front());
if (value.ends_with(suffix))
{
handler();
/// Ensure that no leaders appeared and enable persistent multi-leader mode
/// May fail with ZNOTEMPTY
ops.emplace_back(makeRemoveRequest(path, 0));
ops.emplace_back(makeCreateRequest(path, "", zkutil::CreateMode::Persistent));
/// May fail with ZNODEEXISTS
ops.emplace_back(makeCreateRequest(fs::path(path) / persistent_multiple_leaders, persistent_identifier, zkutil::CreateMode::Persistent));
}
else
{
if (potential_leaders.front() == persistent_multiple_leaders)
return;
/// Ensure that current leader supports multi-leader mode and make it persistent
auto current_leader = fs::path(path) / potential_leaders.front();
Coordination::Stat leader_stat;
String identifier;
if (!zookeeper.tryGet(current_leader, identifier, &leader_stat))
{
LOG_INFO(log, "LeaderElection: leader suddenly changed, will retry");
continue;
}
if (my_node_it == children.begin())
throw Poco::Exception("Assertion failed in LeaderElection");
if (!identifier.ends_with(suffix))
throw Poco::Exception(fmt::format("Found leader replica ({}) with too old version (< 20.6). Stop it before upgrading", identifier));
/// Watch for the node in front of us.
--my_node_it;
std::string get_path_value;
if (!zookeeper.tryGetWatch(path + "/" + *my_node_it, get_path_value, nullptr, task->getWatchCallback()))
task->schedule();
success = true;
}
catch (const KeeperException & e)
{
DB::tryLogCurrentException(log);
if (e.code == Coordination::Error::ZSESSIONEXPIRED)
return;
}
catch (...)
{
DB::tryLogCurrentException(log);
/// Version does not matter, just check that it still exists.
/// May fail with ZNONODE
ops.emplace_back(makeCheckRequest(current_leader, leader_stat.version));
/// May fail with ZNODEEXISTS
ops.emplace_back(makeCreateRequest(fs::path(path) / persistent_multiple_leaders, persistent_identifier, zkutil::CreateMode::Persistent));
}
if (!success)
task->scheduleAfter(10 * 1000);
Coordination::Responses res;
code = zookeeper.tryMulti(ops, res);
if (code == Coordination::Error::ZOK)
return;
else if (code == Coordination::Error::ZNOTEMPTY || code == Coordination::Error::ZNODEEXISTS || code == Coordination::Error::ZNONODE)
LOG_INFO(log, "LeaderElection: leader suddenly changed or new node appeared, will retry");
else
KeeperMultiException::check(code, ops, res);
}
};
using LeaderElectionPtr = std::shared_ptr<LeaderElection>;
throw Poco::Exception("Cannot check that no old leaders exist");
}
}

View File

@ -1,6 +1,7 @@
#include <Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h>
#include <Storages/MergeTree/PinnedPartUUIDs.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Poco/JSON/JSON.h>
#include <Poco/JSON/Object.h>
#include <Poco/JSON/Parser.h>

View File

@ -2,9 +2,9 @@
#include <Storages/StorageReplicatedMergeTree.h>
#include <Poco/Timestamp.h>
#include <Interpreters/Context.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <random>
#include <pcg_random.hpp>
#include <unordered_set>

View File

@ -197,11 +197,6 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup()
updateQuorumIfWeHavePart();
if (storage_settings->replicated_can_become_leader)
storage.enterLeaderElection();
else
LOG_INFO(log, "Will not enter leader election because replicated_can_become_leader=0");
/// Anything above can throw a KeeperException if something is wrong with ZK.
/// Anything below should not throw exceptions.
@ -380,8 +375,6 @@ void ReplicatedMergeTreeRestartingThread::partialShutdown()
LOG_TRACE(log, "Waiting for threads to finish");
storage.exitLeaderElection();
storage.queue_updating_task->deactivate();
storage.mutations_updating_task->deactivate();
storage.mutations_finalizing_task->deactivate();

View File

@ -22,17 +22,13 @@
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Databases/DatabaseReplicatedHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int BAD_ARGUMENTS;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
extern const int NO_ELEMENTS_IN_CONFIG;
extern const int UNKNOWN_STORAGE;
extern const int NO_REPLICA_NAME_GIVEN;
}
@ -62,171 +58,6 @@ static Names extractColumnNames(const ASTPtr & node)
}
}
/** Is used to order Graphite::Retentions by age and precision descending.
* Throws exception if not both age and precision are less or greater then another.
*/
static bool compareRetentions(const Graphite::Retention & a, const Graphite::Retention & b)
{
if (a.age > b.age && a.precision > b.precision)
{
return true;
}
else if (a.age < b.age && a.precision < b.precision)
{
return false;
}
String error_msg = "age and precision should only grow up: "
+ std::to_string(a.age) + ":" + std::to_string(a.precision) + " vs "
+ std::to_string(b.age) + ":" + std::to_string(b.precision);
throw Exception(
error_msg,
ErrorCodes::BAD_ARGUMENTS);
}
/** Read the settings for Graphite rollup from config.
* Example
*
* <graphite_rollup>
* <path_column_name>Path</path_column_name>
* <pattern>
* <regexp>click_cost</regexp>
* <function>any</function>
* <retention>
* <age>0</age>
* <precision>3600</precision>
* </retention>
* <retention>
* <age>86400</age>
* <precision>60</precision>
* </retention>
* </pattern>
* <default>
* <function>max</function>
* <retention>
* <age>0</age>
* <precision>60</precision>
* </retention>
* <retention>
* <age>3600</age>
* <precision>300</precision>
* </retention>
* <retention>
* <age>86400</age>
* <precision>3600</precision>
* </retention>
* </default>
* </graphite_rollup>
*/
static void appendGraphitePattern(
const Poco::Util::AbstractConfiguration & config,
const String & config_element,
Graphite::Patterns & out_patterns,
ContextPtr context)
{
Graphite::Pattern pattern;
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_element, keys);
for (const auto & key : keys)
{
if (key == "regexp")
{
pattern.regexp_str = config.getString(config_element + ".regexp");
pattern.regexp = std::make_shared<OptimizedRegularExpression>(pattern.regexp_str);
}
else if (key == "function")
{
String aggregate_function_name_with_params = config.getString(config_element + ".function");
String aggregate_function_name;
Array params_row;
getAggregateFunctionNameAndParametersArray(
aggregate_function_name_with_params, aggregate_function_name, params_row, "GraphiteMergeTree storage initialization", context);
/// TODO Not only Float64
AggregateFunctionProperties properties;
pattern.function = AggregateFunctionFactory::instance().get(
aggregate_function_name, {std::make_shared<DataTypeFloat64>()}, params_row, properties);
}
else if (startsWith(key, "retention"))
{
pattern.retentions.emplace_back(Graphite::Retention{
.age = config.getUInt(config_element + "." + key + ".age"),
.precision = config.getUInt(config_element + "." + key + ".precision")});
}
else
throw Exception("Unknown element in config: " + key, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
}
if (!pattern.function && pattern.retentions.empty())
throw Exception(
"At least one of an aggregate function or retention rules is mandatory for rollup patterns in GraphiteMergeTree",
ErrorCodes::NO_ELEMENTS_IN_CONFIG);
if (!pattern.function)
{
pattern.type = pattern.TypeRetention;
}
else if (pattern.retentions.empty())
{
pattern.type = pattern.TypeAggregation;
}
else
{
pattern.type = pattern.TypeAll;
}
if (pattern.type & pattern.TypeAggregation) /// TypeAggregation or TypeAll
if (pattern.function->allocatesMemoryInArena())
throw Exception(
"Aggregate function " + pattern.function->getName() + " isn't supported in GraphiteMergeTree", ErrorCodes::NOT_IMPLEMENTED);
/// retention should be in descending order of age.
if (pattern.type & pattern.TypeRetention) /// TypeRetention or TypeAll
std::sort(pattern.retentions.begin(), pattern.retentions.end(), compareRetentions);
out_patterns.emplace_back(pattern);
}
static void setGraphitePatternsFromConfig(ContextPtr context, const String & config_element, Graphite::Params & params)
{
const auto & config = context->getConfigRef();
if (!config.has(config_element))
throw Exception("No '" + config_element + "' element in configuration file", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
params.config_name = config_element;
params.path_column_name = config.getString(config_element + ".path_column_name", "Path");
params.time_column_name = config.getString(config_element + ".time_column_name", "Time");
params.value_column_name = config.getString(config_element + ".value_column_name", "Value");
params.version_column_name = config.getString(config_element + ".version_column_name", "Timestamp");
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_element, keys);
for (const auto & key : keys)
{
if (startsWith(key, "pattern"))
{
appendGraphitePattern(config, config_element + "." + key, params.patterns, context);
}
else if (key == "default")
{
/// See below.
}
else if (key == "path_column_name" || key == "time_column_name" || key == "value_column_name" || key == "version_column_name")
{
/// See above.
}
else
throw Exception("Unknown element in config: " + key, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
}
if (config.has(config_element + ".default"))
appendGraphitePattern(config, config_element + "." + ".default", params.patterns, context);
}
static String getMergeTreeVerboseHelp(bool)
{
using namespace std::string_literals;
@ -542,12 +373,6 @@ static StoragePtr create(const StorageFactory::Arguments & args)
/// to make possible copying metadata files between replicas.
Macros::MacroExpansionInfo info;
info.table_id = args.table_id;
if (is_replicated_database)
{
auto database = DatabaseCatalog::instance().getDatabase(args.table_id.database_name);
info.shard = getReplicatedDatabaseShardName(database);
info.replica = getReplicatedDatabaseReplicaName(database);
}
if (!allow_uuid_macro)
info.table_id.uuid = UUIDHelpers::Nil;
zookeeper_path = args.getContext()->getMacros()->expand(zookeeper_path, info);

View File

@ -32,6 +32,7 @@
#include <Storages/MergeTree/MutateFromLogEntryTask.h>
#include <Storages/VirtualColumnUtils.h>
#include <Storages/MergeTree/MergeTreeReaderCompact.h>
#include <Storages/MergeTree/LeaderElection.h>
#include <Databases/IDatabase.h>
@ -3400,53 +3401,29 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n
}
void StorageReplicatedMergeTree::enterLeaderElection()
void StorageReplicatedMergeTree::startBeingLeader()
{
auto callback = [this]()
if (!getSettings()->replicated_can_become_leader)
{
LOG_INFO(log, "Became leader");
is_leader = true;
merge_selecting_task->activateAndSchedule();
};
try
{
leader_election = std::make_shared<zkutil::LeaderElection>(
getContext()->getSchedulePool(),
fs::path(zookeeper_path) / "leader_election",
*current_zookeeper, /// current_zookeeper lives for the lifetime of leader_election,
/// since before changing `current_zookeeper`, `leader_election` object is destroyed in `partialShutdown` method.
callback,
replica_name);
}
catch (...)
{
leader_election = nullptr;
throw;
LOG_INFO(log, "Will not enter leader election because replicated_can_become_leader=0");
return;
}
zkutil::checkNoOldLeaders(log, *current_zookeeper, fs::path(zookeeper_path) / "leader_election");
LOG_INFO(log, "Became leader");
is_leader = true;
merge_selecting_task->activateAndSchedule();
}
void StorageReplicatedMergeTree::exitLeaderElection()
void StorageReplicatedMergeTree::stopBeingLeader()
{
if (!leader_election)
if (!is_leader)
return;
/// Shut down the leader election thread to avoid suddenly becoming the leader again after
/// we have stopped the merge_selecting_thread, but before we have deleted the leader_election object.
leader_election->shutdown();
if (is_leader)
{
LOG_INFO(log, "Stopped being leader");
is_leader = false;
merge_selecting_task->deactivate();
}
/// Delete the node in ZK only after we have stopped the merge_selecting_thread - so that only one
/// replica assigns merges at any given time.
leader_election = nullptr;
LOG_INFO(log, "Stopped being leader");
is_leader = false;
merge_selecting_task->deactivate();
}
ConnectionTimeouts StorageReplicatedMergeTree::getFetchPartHTTPTimeouts(ContextPtr local_context)
@ -4109,10 +4086,12 @@ void StorageReplicatedMergeTree::startup()
assert(prev_ptr == nullptr);
getContext()->getInterserverIOHandler().addEndpoint(data_parts_exchange_ptr->getId(replica_path), data_parts_exchange_ptr);
startBeingLeader();
/// In this thread replica will be activated.
restarting_thread.start();
/// Wait while restarting_thread initializes LeaderElection (and so on) or makes first attempt to do it
/// Wait while restarting_thread finishing initialization
startup_event.wait();
startBackgroundMovesIfNeeded();
@ -4145,6 +4124,7 @@ void StorageReplicatedMergeTree::shutdown()
fetcher.blocker.cancelForever();
merger_mutator.merges_blocker.cancelForever();
parts_mover.moves_blocker.cancelForever();
stopBeingLeader();
restarting_thread.shutdown();
background_operations_assignee.finish();

View File

@ -19,7 +19,6 @@
#include <Storages/MergeTree/EphemeralLockInZooKeeper.h>
#include <Storages/MergeTree/DataPartsExchange.h>
#include <Storages/MergeTree/ReplicatedMergeTreeAddress.h>
#include <Storages/MergeTree/LeaderElection.h>
#include <Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h>
#include <Storages/MergeTree/FutureMergedMutatedPart.h>
#include <Storages/MergeTree/MergeFromLogEntryTask.h>
@ -320,7 +319,6 @@ private:
* It can be false only when old ClickHouse versions are working on the same cluster, because now we allow multiple leaders.
*/
std::atomic<bool> is_leader {false};
zkutil::LeaderElectionPtr leader_election;
InterserverIOEndpointPtr data_parts_exchange_endpoint;
@ -514,15 +512,10 @@ private:
bool processQueueEntry(ReplicatedMergeTreeQueue::SelectedEntryPtr entry);
/// Postcondition:
/// either leader_election is fully initialized (node in ZK is created and the watching thread is launched)
/// or an exception is thrown and leader_election is destroyed.
void enterLeaderElection();
/// Postcondition:
/// is_leader is false, merge_selecting_thread is stopped, leader_election is nullptr.
/// leader_election node in ZK is either deleted, or the session is marked expired.
void exitLeaderElection();
/// Start being leader (if not disabled by setting).
/// Since multi-leaders are allowed, it just sets is_leader flag.
void startBeingLeader();
void stopBeingLeader();
/** Selects the parts to merge and writes to the log.
*/

View File

@ -10,6 +10,7 @@ NamesAndTypesList StorageSystemGraphite::getNamesAndTypes()
{
return {
{"config_name", std::make_shared<DataTypeString>()},
{"rule_type", std::make_shared<DataTypeString>()},
{"regexp", std::make_shared<DataTypeString>()},
{"function", std::make_shared<DataTypeString>()},
{"age", std::make_shared<DataTypeUInt64>()},
@ -85,6 +86,7 @@ void StorageSystemGraphite::fillData(MutableColumns & res_columns, ContextPtr co
bool is_default = pattern.regexp == nullptr;
String regexp;
String function;
const String & rule_type = ruleTypeStr(pattern.rule_type);
if (is_default)
{
@ -107,6 +109,7 @@ void StorageSystemGraphite::fillData(MutableColumns & res_columns, ContextPtr co
{
size_t i = 0;
res_columns[i++]->insert(config.first);
res_columns[i++]->insert(rule_type);
res_columns[i++]->insert(regexp);
res_columns[i++]->insert(function);
res_columns[i++]->insert(retention.age);
@ -121,6 +124,7 @@ void StorageSystemGraphite::fillData(MutableColumns & res_columns, ContextPtr co
{
size_t i = 0;
res_columns[i++]->insert(config.first);
res_columns[i++]->insert(rule_type);
res_columns[i++]->insert(regexp);
res_columns[i++]->insert(function);
res_columns[i++]->insertDefault();

View File

@ -205,26 +205,31 @@ def get_stacktraces_from_gdb(server_pid):
# collect server stacktraces from system.stack_trace table
# it does not work in Sandbox
def get_stacktraces_from_clickhouse(client, replicated_database=False):
def get_stacktraces_from_clickhouse(args):
settings_str = ' '.join([
get_additional_client_options(args),
'--allow_introspection_functions=1',
'--skip_unavailable_shards=1',
])
replicated_msg = \
"{} --allow_introspection_functions=1 --skip_unavailable_shards=1 --query \
"{} {} --query \
\"SELECT materialize((hostName(), tcpPort())) as host, thread_id, \
arrayStringConcat(arrayMap(x, y -> concat(x, ': ', y), \
arrayMap(x -> addressToLine(x), trace), \
arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace \
FROM clusterAllReplicas('test_cluster_database_replicated', 'system.stack_trace') \
ORDER BY host, thread_id FORMAT Vertical\"".format(client)
ORDER BY host, thread_id FORMAT Vertical\"".format(args.client, settings_str)
msg = \
"{} --allow_introspection_functions=1 --query \
"{} {} --query \
\"SELECT arrayStringConcat(arrayMap(x, y -> concat(x, ': ', y), \
arrayMap(x -> addressToLine(x), trace), \
arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace \
FROM system.stack_trace FORMAT Vertical\"".format(client)
FROM system.stack_trace FORMAT Vertical\"".format(args.client, settings_str)
try:
return subprocess.check_output(
replicated_msg if replicated_database else msg,
replicated_msg if args.replicated_database else msg,
shell=True, stderr=subprocess.STDOUT).decode('utf-8')
except Exception as e:
print(f"Error occurred while receiving stack traces from client: {e}")
@ -250,8 +255,7 @@ def print_stacktraces() -> None:
if bt is None:
print("\nCollecting stacktraces from system.stacktraces table:")
bt = get_stacktraces_from_clickhouse(
args.client, args.replicated_database)
bt = get_stacktraces_from_clickhouse(args)
if bt is not None:
print(bt)

View File

@ -100,3 +100,19 @@ def exec_query_with_retry(instance, query, retry_count=40, sleep_time=0.5, silen
time.sleep(sleep_time)
else:
raise exception
def csv_compare(result, expected):
csv_result = TSV(result)
csv_expected = TSV(expected)
mismatch = []
max_len = len(csv_result) if len(csv_result) > len(csv_expected) else len(csv_expected)
for i in range(max_len):
if i >= len(csv_result):
mismatch.append("-[%d]=%s" % (i, csv_expected.lines[i]))
elif i >= len(csv_expected):
mismatch.append("+[%d]=%s" % (i, csv_result.lines[i]))
elif csv_expected.lines[i] != csv_result.lines[i]:
mismatch.append("-[%d]=%s" % (i, csv_expected.lines[i]))
mismatch.append("+[%d]=%s" % (i, csv_result.lines[i]))
return "\n".join(mismatch)

View File

@ -11,13 +11,14 @@ node2 = cluster.add_instance('node2', main_configs=['configs/wide_parts_only.xml
def start_cluster():
try:
cluster.start()
for i, node in enumerate([node1, node2]):
node.query_with_retry(
'''CREATE TABLE t(date Date, id UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/t', '{}')
PARTITION BY toYYYYMM(date)
ORDER BY id'''.format(i))
create_query = '''CREATE TABLE t(date Date, id UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/t', '{}')
PARTITION BY toYYYYMM(date)
ORDER BY id'''
node1.query(create_query.format(1))
node1.query("DETACH TABLE t") # stop being leader
node2.query(create_query.format(2))
node1.query("ATTACH TABLE t")
yield cluster
finally:

View File

@ -6,6 +6,7 @@ import pytest
from helpers.client import QueryRuntimeException
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
from helpers.test_tools import csv_compare
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance',
@ -234,18 +235,19 @@ SELECT * FROM test.graphite;
def test_system_graphite_retentions(graphite_table):
expected = '''
graphite_rollup \\\\.count$ sum 0 0 1 0 ['test'] ['graphite']
graphite_rollup \\\\.max$ max 0 0 2 0 ['test'] ['graphite']
graphite_rollup ^five_min\\\\. 31536000 14400 3 0 ['test'] ['graphite']
graphite_rollup ^five_min\\\\. 5184000 3600 3 0 ['test'] ['graphite']
graphite_rollup ^five_min\\\\. 0 300 3 0 ['test'] ['graphite']
graphite_rollup ^one_min avg 31536000 600 4 0 ['test'] ['graphite']
graphite_rollup ^one_min avg 7776000 300 4 0 ['test'] ['graphite']
graphite_rollup ^one_min avg 0 60 4 0 ['test'] ['graphite']
graphite_rollup all \\\\.count$ sum 0 0 1 0 ['test'] ['graphite']
graphite_rollup all \\\\.max$ max 0 0 2 0 ['test'] ['graphite']
graphite_rollup all ^five_min\\\\. 31536000 14400 3 0 ['test'] ['graphite']
graphite_rollup all ^five_min\\\\. 5184000 3600 3 0 ['test'] ['graphite']
graphite_rollup all ^five_min\\\\. 0 300 3 0 ['test'] ['graphite']
graphite_rollup all ^one_min avg 31536000 600 4 0 ['test'] ['graphite']
graphite_rollup all ^one_min avg 7776000 300 4 0 ['test'] ['graphite']
graphite_rollup all ^one_min avg 0 60 4 0 ['test'] ['graphite']
'''
result = q('SELECT * from system.graphite_retentions')
assert TSV(result) == TSV(expected)
mismatch = csv_compare(result, expected)
assert len(mismatch) == 0, f"got\n{result}\nwant\n{expected}\ndiff\n{mismatch}\n"
q('''
DROP TABLE IF EXISTS test.graphite2;

View File

@ -0,0 +1,120 @@
<clickhouse>
<!-- retention scheme for GraphiteMergeTree engine-->
<graphite_rollup>
<path_column_name>metric</path_column_name>
<time_column_name>timestamp</time_column_name>
<value_column_name>value</value_column_name>
<version_column_name>updated</version_column_name>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.count$</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.max$</regexp>
<function>max</function>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>^five_min\.</regexp>
<retention>
<age>0</age>
<precision>300</precision>
</retention>
<retention>
<age>5184000</age>
<precision>3600</precision>
</retention>
<retention>
<age>31536000</age>
<precision>14400</precision>
</retention>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>^one_min</regexp>
<function>avg</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>7776000</age>
<precision>300</precision>
</retention>
<retention>
<age>31536000</age>
<precision>600</precision>
</retention>
</pattern>
<pattern>
<rule_type>tagged</rule_type>
<regexp><![CDATA[[\?&]retention=one_min(&.*)?$]]></regexp>
<function>avg</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>7776000</age>
<precision>300</precision>
</retention>
<retention>
<age>31536000</age>
<precision>600</precision>
</retention>
</pattern>
<pattern>
<rule_type>tag_list</rule_type>
<regexp>retention=five_min</regexp>
<function>avg</function>
<retention>
<age>0</age>
<precision>300</precision>
</retention>
<retention>
<age>5184000</age>
<precision>3600</precision>
</retention>
<retention>
<age>31536000</age>
<precision>14400</precision>
</retention>
</pattern>
<pattern>
<rule_type>tagged</rule_type>
<regexp>^for_taggged</regexp>
<function>avg</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>7776000</age>
<precision>300</precision>
</retention>
<retention>
<age>31536000</age>
<precision>600</precision>
</retention>
</pattern>
<pattern>
<rule_type>all</rule_type>
<regexp>^ten_min\.</regexp>
<function>sum</function>
<retention>
<age>0</age>
<precision>600</precision>
</retention>
<retention>
<age>5184000</age>
<precision>7200</precision>
</retention>
<retention>
<age>31536000</age>
<precision>28800</precision>
</retention>
</pattern>
</graphite_rollup>
</clickhouse>

View File

@ -0,0 +1,8 @@
<?xml version="1.0"?>
<clickhouse>
<profiles>
<default>
<optimize_on_insert>0</optimize_on_insert>
</default>
</profiles>
</clickhouse>

View File

@ -0,0 +1,580 @@
import datetime
import os.path as p
import time
import sys
import pytest
from helpers.client import QueryRuntimeException
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
from helpers.test_tools import csv_compare
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance',
main_configs=['configs/graphite_rollup.xml'],
user_configs=["configs/users.xml"])
q = instance.query
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
q('CREATE DATABASE test')
yield cluster
finally:
cluster.shutdown()
@pytest.fixture
def graphite_table(started_cluster):
q('''
DROP TABLE IF EXISTS test.graphite;
CREATE TABLE test.graphite
(metric String, value Float64, timestamp UInt32, date Date, updated UInt32)
ENGINE = GraphiteMergeTree('graphite_rollup')
PARTITION BY toYYYYMM(date)
ORDER BY (metric, timestamp)
SETTINGS index_granularity=8192;
''')
yield
q('DROP TABLE test.graphite')
def test_rollup_versions_plain(graphite_table):
timestamp = int(time.time())
rounded_timestamp = timestamp - timestamp % 60
date = datetime.date.today().isoformat()
# Insert rows with timestamps relative to the current time so that the
# first retention clause is active.
# Two parts are created.
q('''
INSERT INTO test.graphite (metric, value, timestamp, date, updated)
VALUES ('one_min.x1', 100, {timestamp}, '{date}', 1);
INSERT INTO test.graphite (metric, value, timestamp, date, updated)
VALUES ('one_min.x1', 200, {timestamp}, '{date}', 2);
'''.format(timestamp=timestamp, date=date))
expected1 = '''\
one_min.x1 100 {timestamp} {date} 1
one_min.x1 200 {timestamp} {date} 2
'''.format(timestamp=timestamp, date=date)
assert TSV(
q('SELECT * FROM test.graphite ORDER BY updated')
) == TSV(expected1)
q('OPTIMIZE TABLE test.graphite')
# After rollup only the row with max version is retained.
expected2 = '''\
one_min.x1 200 {timestamp} {date} 2
'''.format(timestamp=rounded_timestamp, date=date)
assert TSV(q('SELECT * FROM test.graphite')) == TSV(expected2)
def test_rollup_versions_tagged(graphite_table):
timestamp = int(time.time())
rounded_timestamp = timestamp - timestamp % 60
date = datetime.date.today().isoformat()
# Insert rows with timestamps relative to the current time so that the
# first retention clause is active.
# Two parts are created.
q('''
INSERT INTO test.graphite (metric, value, timestamp, date, updated)
VALUES ('x1?retention=one_min', 100, {timestamp}, '{date}', 1);
INSERT INTO test.graphite (metric, value, timestamp, date, updated)
VALUES ('x1?retention=one_min', 200, {timestamp}, '{date}', 2);
'''.format(timestamp=timestamp, date=date))
expected1 = '''\
x1?retention=one_min 100 {timestamp} {date} 1
x1?retention=one_min 200 {timestamp} {date} 2
'''.format(timestamp=timestamp, date=date)
result = q('SELECT * FROM test.graphite ORDER BY metric, updated')
mismatch = csv_compare(result, expected1)
assert len(mismatch) == 0, f"got\n{result}\nwant\n{expected1}\ndiff\n{mismatch}\n"
q('OPTIMIZE TABLE test.graphite')
# After rollup only the row with max version is retained.
expected2 = '''\
x1?retention=one_min 200 {timestamp} {date} 2
'''.format(timestamp=rounded_timestamp, date=date)
result = q('SELECT * FROM test.graphite ORDER BY metric, updated')
mismatch = csv_compare(result, expected2)
assert len(mismatch) == 0, f"got\n{result}\nwant\n{expected2}\ndiff\n{mismatch}\n"
def test_rollup_versions_all(graphite_table):
timestamp = int(time.time())
rounded_timestamp = timestamp - timestamp % 600
date = datetime.date.today().isoformat()
# Insert rows with timestamps relative to the current time so that the
# first retention clause is active.
# Two parts are created.
q('''
INSERT INTO test.graphite (metric, value, timestamp, date, updated)
VALUES ('ten_min.x1', 100, {timestamp}, '{date}', 1);
INSERT INTO test.graphite (metric, value, timestamp, date, updated)
VALUES ('ten_min.x1', 200, {timestamp}, '{date}', 2);
INSERT INTO test.graphite (metric, value, timestamp, date, updated)
VALUES ('ten_min.x1?env=staging', 100, {timestamp}, '{date}', 1);
INSERT INTO test.graphite (metric, value, timestamp, date, updated)
VALUES ('ten_min.x1?env=staging', 200, {timestamp}, '{date}', 2);
'''.format(timestamp=timestamp, date=date))
expected1 = '''\
ten_min.x1 100 {timestamp} {date} 1
ten_min.x1 200 {timestamp} {date} 2
ten_min.x1?env=staging 100 {timestamp} {date} 1
ten_min.x1?env=staging 200 {timestamp} {date} 2
'''.format(timestamp=timestamp, date=date)
result = q('SELECT * FROM test.graphite ORDER BY metric, updated')
mismatch = csv_compare(result, expected1)
assert len(mismatch) == 0, f"got\n{result}\nwant\n{expected1}\ndiff\n{mismatch}\n"
q('OPTIMIZE TABLE test.graphite')
# After rollup only the row with max version is retained.
expected2 = '''\
ten_min.x1 200 {timestamp} {date} 2
ten_min.x1?env=staging 200 {timestamp} {date} 2
'''.format(timestamp=rounded_timestamp, date=date)
result = q('SELECT * FROM test.graphite ORDER BY metric, updated')
mismatch = csv_compare(result, expected2)
assert len(mismatch) == 0, f"got\n{result}\nwant\n{expected2}\ndiff\n{mismatch}\n"
def test_rollup_aggregation_plain(graphite_table):
# This query essentially emulates what rollup does.
result1 = q('''
SELECT avg(v), max(upd)
FROM (SELECT timestamp,
argMax(value, (updated, number)) AS v,
max(updated) AS upd
FROM (SELECT 'one_min.x5' AS metric,
toFloat64(number) AS value,
toUInt32(1111111111 + intDiv(number, 3)) AS timestamp,
toDate('2017-02-02') AS date,
toUInt32(intDiv(number, 2)) AS updated,
number
FROM system.numbers LIMIT 1000000)
WHERE intDiv(timestamp, 600) * 600 = 1111444200
GROUP BY timestamp)
''')
expected1 = '''\
999634.9918367347 499999
'''
assert TSV(result1) == TSV(expected1)
# Timestamp 1111111111 is in sufficiently distant past
# so that the last retention clause is active.
result2 = q('''
INSERT INTO test.graphite
SELECT 'one_min.x' AS metric,
toFloat64(number) AS value,
toUInt32(1111111111 + intDiv(number, 3)) AS timestamp,
toDate('2017-02-02') AS date, toUInt32(intDiv(number, 2)) AS updated
FROM (SELECT * FROM system.numbers LIMIT 1000000)
WHERE intDiv(timestamp, 600) * 600 = 1111444200;
OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL;
SELECT * FROM test.graphite;
''')
expected2 = '''\
one_min.x 999634.9918367347 1111444200 2017-02-02 499999
'''
assert TSV(result2) == TSV(expected2)
def test_rollup_aggregation_tagged(graphite_table):
# This query essentially emulates what rollup does.
result1 = q('''
SELECT avg(v), max(upd)
FROM (SELECT timestamp,
argMax(value, (updated, number)) AS v,
max(updated) AS upd
FROM (SELECT 'x?retention=one_min' AS metric,
toFloat64(number) AS value,
toUInt32(1111111111 + intDiv(number, 3)) AS timestamp,
toDate('2017-02-02') AS date,
toUInt32(intDiv(number, 2)) AS updated,
number
FROM system.numbers LIMIT 1000000)
WHERE intDiv(timestamp, 600) * 600 = 1111444200
GROUP BY timestamp)
''')
expected1 = '''\
999634.9918367347 499999
'''
assert TSV(result1) == TSV(expected1)
# Timestamp 1111111111 is in sufficiently distant past
# so that the last retention clause is active.
result2 = q('''
INSERT INTO test.graphite
SELECT 'x?retention=one_min' AS metric,
toFloat64(number) AS value,
toUInt32(1111111111 + intDiv(number, 3)) AS timestamp,
toDate('2017-02-02') AS date, toUInt32(intDiv(number, 2)) AS updated
FROM (SELECT * FROM system.numbers LIMIT 1000000)
WHERE intDiv(timestamp, 600) * 600 = 1111444200;
OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL;
SELECT * FROM test.graphite;
''')
expected2 = '''\
x?retention=one_min 999634.9918367347 1111444200 2017-02-02 499999
'''
assert TSV(result2) == TSV(expected2)
def test_rollup_aggregation_2_plain(graphite_table):
result = q('''
INSERT INTO test.graphite
SELECT 'one_min.x' AS metric,
toFloat64(number) AS value,
toUInt32(1111111111 - intDiv(number, 3)) AS timestamp,
toDate('2017-02-02') AS date,
toUInt32(100 - number) AS updated
FROM (SELECT * FROM system.numbers LIMIT 50);
OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL;
SELECT * FROM test.graphite;
''')
expected = '''\
one_min.x 24 1111110600 2017-02-02 100
'''
assert TSV(result) == TSV(expected)
def test_rollup_aggregation_2_tagged(graphite_table):
result = q('''
INSERT INTO test.graphite
SELECT 'x?retention=one_min' AS metric,
toFloat64(number) AS value,
toUInt32(1111111111 - intDiv(number, 3)) AS timestamp,
toDate('2017-02-02') AS date,
toUInt32(100 - number) AS updated
FROM (SELECT * FROM system.numbers LIMIT 50);
OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL;
SELECT * FROM test.graphite;
''')
expected = '''\
x?retention=one_min 24 1111110600 2017-02-02 100
'''
assert TSV(result) == TSV(expected)
def test_multiple_paths_and_versions_plain(graphite_table):
result = q('''
INSERT INTO test.graphite
SELECT 'one_min.x' AS metric,
toFloat64(number) AS value,
toUInt32(1111111111 + intDiv(number, 3) * 600) AS timestamp,
toDate('2017-02-02') AS date,
toUInt32(100 - number) AS updated
FROM (SELECT * FROM system.numbers LIMIT 50);
OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL;
SELECT * FROM test.graphite;
INSERT INTO test.graphite
SELECT 'one_min.y' AS metric,
toFloat64(number) AS value,
toUInt32(1111111111 + number * 600) AS timestamp,
toDate('2017-02-02') AS date,
toUInt32(100 - number) AS updated
FROM (SELECT * FROM system.numbers LIMIT 50);
OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL;
SELECT * FROM test.graphite;
''')
with open(p.join(p.dirname(__file__),
'test_multiple_paths_and_versions.reference.plain')
) as reference:
assert TSV(result) == TSV(reference)
def test_multiple_paths_and_versions_tagged(graphite_table):
result = q('''
INSERT INTO test.graphite
SELECT 'x?retention=one_min' AS metric,
toFloat64(number) AS value,
toUInt32(1111111111 + intDiv(number, 3) * 600) AS timestamp,
toDate('2017-02-02') AS date,
toUInt32(100 - number) AS updated
FROM (SELECT * FROM system.numbers LIMIT 50);
OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL;
SELECT * FROM test.graphite;
INSERT INTO test.graphite
SELECT 'y?retention=one_min' AS metric,
toFloat64(number) AS value,
toUInt32(1111111111 + number * 600) AS timestamp,
toDate('2017-02-02') AS date,
toUInt32(100 - number) AS updated
FROM (SELECT * FROM system.numbers LIMIT 50);
OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL;
SELECT * FROM test.graphite;
''')
with open(p.join(p.dirname(__file__),
'test_multiple_paths_and_versions.reference.tagged')
) as reference:
assert TSV(result) == TSV(reference)
def test_multiple_output_blocks(graphite_table):
MERGED_BLOCK_SIZE = 8192
to_insert = ''
expected = ''
for i in range(2 * MERGED_BLOCK_SIZE + 1):
rolled_up_time = 1000000200 + 600 * i
for j in range(3):
cur_time = rolled_up_time + 100 * j
to_insert += 'one_min.x1 {} {} 2001-09-09 1\n'.format(
10 * j, cur_time
)
to_insert += 'one_min.x1 {} {} 2001-09-09 2\n'.format(
10 * (j + 1), cur_time
)
expected += 'one_min.x1 20 {} 2001-09-09 2\n'.format(rolled_up_time)
q('INSERT INTO test.graphite FORMAT TSV', to_insert)
result = q('''
OPTIMIZE TABLE test.graphite PARTITION 200109 FINAL;
SELECT * FROM test.graphite;
''')
assert TSV(result) == TSV(expected)
def test_paths_not_matching_any_pattern(graphite_table):
to_insert = '''\
one_min.x1 100 1000000000 2001-09-09 1
zzzzzzzz 100 1000000001 2001-09-09 1
zzzzzzzz 200 1000000001 2001-09-09 2
'''
q('INSERT INTO test.graphite FORMAT TSV', to_insert)
expected = '''\
one_min.x1 100 999999600 2001-09-09 1
zzzzzzzz 200 1000000001 2001-09-09 2
'''
result = q('''
OPTIMIZE TABLE test.graphite PARTITION 200109 FINAL;
SELECT * FROM test.graphite;
''')
assert TSV(result) == TSV(expected)
def test_rules_isolation(graphite_table):
to_insert = '''\
one_min.x1 100 1000000000 2001-09-09 1
for_taggged 100 1000000001 2001-09-09 1
for_taggged 200 1000000001 2001-09-09 2
one_min?env=staging 100 1000000001 2001-09-09 1
one_min?env=staging 200 1000000001 2001-09-09 2
'''
q('INSERT INTO test.graphite FORMAT TSV', to_insert)
expected = '''\
for_taggged 200 1000000001 2001-09-09 2
one_min.x1 100 999999600 2001-09-09 1
one_min?env=staging 200 1000000001 2001-09-09 2
'''
result = q('''
OPTIMIZE TABLE test.graphite PARTITION 200109 FINAL;
SELECT * FROM test.graphite;
''')
result = q('SELECT * FROM test.graphite ORDER BY metric, updated')
mismatch = csv_compare(result, expected)
assert len(mismatch) == 0, f"got\n{result}\nwant\n{expected}\ndiff\n{mismatch}\n"
def test_system_graphite_retentions(graphite_table):
expected = '''
graphite_rollup plain \\\\.count$ sum 0 0 1 0 ['test'] ['graphite']
graphite_rollup plain \\\\.max$ max 0 0 2 0 ['test'] ['graphite']
graphite_rollup plain ^five_min\\\\. 31536000 14400 3 0 ['test'] ['graphite']
graphite_rollup plain ^five_min\\\\. 5184000 3600 3 0 ['test'] ['graphite']
graphite_rollup plain ^five_min\\\\. 0 300 3 0 ['test'] ['graphite']
graphite_rollup plain ^one_min avg 31536000 600 4 0 ['test'] ['graphite']
graphite_rollup plain ^one_min avg 7776000 300 4 0 ['test'] ['graphite']
graphite_rollup plain ^one_min avg 0 60 4 0 ['test'] ['graphite']
graphite_rollup tagged [\\\\?&]retention=one_min(&.*)?$ avg 31536000 600 5 0 ['test'] ['graphite']
graphite_rollup tagged [\\\\?&]retention=one_min(&.*)?$ avg 7776000 300 5 0 ['test'] ['graphite']
graphite_rollup tagged [\\\\?&]retention=one_min(&.*)?$ avg 0 60 5 0 ['test'] ['graphite']
graphite_rollup tagged [\\\\?&]retention=five_min(&.*)?$ avg 31536000 14400 6 0 ['test'] ['graphite']
graphite_rollup tagged [\\\\?&]retention=five_min(&.*)?$ avg 5184000 3600 6 0 ['test'] ['graphite']
graphite_rollup tagged [\\\\?&]retention=five_min(&.*)?$ avg 0 300 6 0 ['test'] ['graphite']
graphite_rollup tagged ^for_taggged avg 31536000 600 7 0 ['test'] ['graphite']
graphite_rollup tagged ^for_taggged avg 7776000 300 7 0 ['test'] ['graphite']
graphite_rollup tagged ^for_taggged avg 0 60 7 0 ['test'] ['graphite']
graphite_rollup all ^ten_min\\\\. sum 31536000 28800 8 0 ['test'] ['graphite']
graphite_rollup all ^ten_min\\\\. sum 5184000 7200 8 0 ['test'] ['graphite']
graphite_rollup all ^ten_min\\\\. sum 0 600 8 0 ['test'] ['graphite']
'''
result = q('SELECT * from system.graphite_retentions')
mismatch = csv_compare(result, expected)
assert len(mismatch) == 0, f"got\n{result}\nwant\n{expected}\ndiff\n{mismatch}\n"
q('''
DROP TABLE IF EXISTS test.graphite2;
CREATE TABLE test.graphite2
(metric String, value Float64, timestamp UInt32, date Date, updated UInt32)
ENGINE = GraphiteMergeTree('graphite_rollup')
PARTITION BY toYYYYMM(date)
ORDER BY (metric, timestamp)
SETTINGS index_granularity=8192;
''')
expected = '''
graphite_rollup ['test','test'] ['graphite','graphite2']
graphite_rollup ['test','test'] ['graphite','graphite2']
graphite_rollup ['test','test'] ['graphite','graphite2']
graphite_rollup ['test','test'] ['graphite','graphite2']
graphite_rollup ['test','test'] ['graphite','graphite2']
graphite_rollup ['test','test'] ['graphite','graphite2']
graphite_rollup ['test','test'] ['graphite','graphite2']
graphite_rollup ['test','test'] ['graphite','graphite2']
'''
result = q('''
SELECT
config_name,
Tables.database,
Tables.table
FROM system.graphite_retentions
''')
assert csv_compare(result, expected), f"got\n{result}\nwant\n{expected}"
def test_path_dangling_pointer(graphite_table):
q('''
DROP TABLE IF EXISTS test.graphite2;
CREATE TABLE test.graphite2
(metric String, value Float64, timestamp UInt32, date Date, updated UInt32)
ENGINE = GraphiteMergeTree('graphite_rollup')
PARTITION BY toYYYYMM(date)
ORDER BY (metric, timestamp)
SETTINGS index_granularity=1;
''')
path = 'abcd' * 4000000 # 16MB
q('INSERT INTO test.graphite2 FORMAT TSV',
"{}\t0.0\t0\t2018-01-01\t100\n".format(path))
q('INSERT INTO test.graphite2 FORMAT TSV',
"{}\t0.0\t0\t2018-01-01\t101\n".format(path))
for version in range(10):
q('INSERT INTO test.graphite2 FORMAT TSV',
"{}\t0.0\t0\t2018-01-01\t{}\n".format(path, version))
while True:
q('OPTIMIZE TABLE test.graphite2 PARTITION 201801 FINAL')
parts = int(q("SELECT count() FROM system.parts "
"WHERE active AND database='test' "
"AND table='graphite2'"))
if parts == 1:
break
print(('Parts', parts))
assert TSV(
q("SELECT value, timestamp, date, updated FROM test.graphite2")
) == TSV("0\t0\t2018-01-01\t101\n")
q('DROP TABLE test.graphite2')
def test_combined_rules(graphite_table):
# 1487970000 ~ Sat 25 Feb 00:00:00 MSK 2017
to_insert = 'INSERT INTO test.graphite VALUES '
expected_unmerged = ''
for i in range(384):
to_insert += "('five_min.count', {v}, {t}, toDate({t}), 1), ".format(
v=1, t=1487970000 + (i * 300)
)
to_insert += "('five_min.max', {v}, {t}, toDate({t}), 1), ".format(
v=i, t=1487970000 + (i * 300)
)
expected_unmerged += ("five_min.count\t{v1}\t{t}\n"
"five_min.max\t{v2}\t{t}\n").format(
v1=1, v2=i,
t=1487970000 + (i * 300)
)
q(to_insert)
assert TSV(q('SELECT metric, value, timestamp FROM test.graphite'
' ORDER BY (timestamp, metric)')) == TSV(expected_unmerged)
q('OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL')
expected_merged = '''
five_min.count 48 1487970000 2017-02-25 1
five_min.count 48 1487984400 2017-02-25 1
five_min.count 48 1487998800 2017-02-25 1
five_min.count 48 1488013200 2017-02-25 1
five_min.count 48 1488027600 2017-02-25 1
five_min.count 48 1488042000 2017-02-25 1
five_min.count 48 1488056400 2017-02-26 1
five_min.count 48 1488070800 2017-02-26 1
five_min.max 47 1487970000 2017-02-25 1
five_min.max 95 1487984400 2017-02-25 1
five_min.max 143 1487998800 2017-02-25 1
five_min.max 191 1488013200 2017-02-25 1
five_min.max 239 1488027600 2017-02-25 1
five_min.max 287 1488042000 2017-02-25 1
five_min.max 335 1488056400 2017-02-26 1
five_min.max 383 1488070800 2017-02-26 1
'''
assert TSV(q('SELECT * FROM test.graphite'
' ORDER BY (metric, timestamp)')) == TSV(expected_merged)

View File

@ -0,0 +1,84 @@
one_min.x 0 1111110600 2017-02-02 100
one_min.x 3 1111111200 2017-02-02 97
one_min.x 6 1111111800 2017-02-02 94
one_min.x 9 1111112400 2017-02-02 91
one_min.x 12 1111113000 2017-02-02 88
one_min.x 15 1111113600 2017-02-02 85
one_min.x 18 1111114200 2017-02-02 82
one_min.x 21 1111114800 2017-02-02 79
one_min.x 24 1111115400 2017-02-02 76
one_min.x 27 1111116000 2017-02-02 73
one_min.x 30 1111116600 2017-02-02 70
one_min.x 33 1111117200 2017-02-02 67
one_min.x 36 1111117800 2017-02-02 64
one_min.x 39 1111118400 2017-02-02 61
one_min.x 42 1111119000 2017-02-02 58
one_min.x 45 1111119600 2017-02-02 55
one_min.x 48 1111120200 2017-02-02 52
one_min.x 0 1111110600 2017-02-02 100
one_min.x 3 1111111200 2017-02-02 97
one_min.x 6 1111111800 2017-02-02 94
one_min.x 9 1111112400 2017-02-02 91
one_min.x 12 1111113000 2017-02-02 88
one_min.x 15 1111113600 2017-02-02 85
one_min.x 18 1111114200 2017-02-02 82
one_min.x 21 1111114800 2017-02-02 79
one_min.x 24 1111115400 2017-02-02 76
one_min.x 27 1111116000 2017-02-02 73
one_min.x 30 1111116600 2017-02-02 70
one_min.x 33 1111117200 2017-02-02 67
one_min.x 36 1111117800 2017-02-02 64
one_min.x 39 1111118400 2017-02-02 61
one_min.x 42 1111119000 2017-02-02 58
one_min.x 45 1111119600 2017-02-02 55
one_min.x 48 1111120200 2017-02-02 52
one_min.y 0 1111110600 2017-02-02 100
one_min.y 1 1111111200 2017-02-02 99
one_min.y 2 1111111800 2017-02-02 98
one_min.y 3 1111112400 2017-02-02 97
one_min.y 4 1111113000 2017-02-02 96
one_min.y 5 1111113600 2017-02-02 95
one_min.y 6 1111114200 2017-02-02 94
one_min.y 7 1111114800 2017-02-02 93
one_min.y 8 1111115400 2017-02-02 92
one_min.y 9 1111116000 2017-02-02 91
one_min.y 10 1111116600 2017-02-02 90
one_min.y 11 1111117200 2017-02-02 89
one_min.y 12 1111117800 2017-02-02 88
one_min.y 13 1111118400 2017-02-02 87
one_min.y 14 1111119000 2017-02-02 86
one_min.y 15 1111119600 2017-02-02 85
one_min.y 16 1111120200 2017-02-02 84
one_min.y 17 1111120800 2017-02-02 83
one_min.y 18 1111121400 2017-02-02 82
one_min.y 19 1111122000 2017-02-02 81
one_min.y 20 1111122600 2017-02-02 80
one_min.y 21 1111123200 2017-02-02 79
one_min.y 22 1111123800 2017-02-02 78
one_min.y 23 1111124400 2017-02-02 77
one_min.y 24 1111125000 2017-02-02 76
one_min.y 25 1111125600 2017-02-02 75
one_min.y 26 1111126200 2017-02-02 74
one_min.y 27 1111126800 2017-02-02 73
one_min.y 28 1111127400 2017-02-02 72
one_min.y 29 1111128000 2017-02-02 71
one_min.y 30 1111128600 2017-02-02 70
one_min.y 31 1111129200 2017-02-02 69
one_min.y 32 1111129800 2017-02-02 68
one_min.y 33 1111130400 2017-02-02 67
one_min.y 34 1111131000 2017-02-02 66
one_min.y 35 1111131600 2017-02-02 65
one_min.y 36 1111132200 2017-02-02 64
one_min.y 37 1111132800 2017-02-02 63
one_min.y 38 1111133400 2017-02-02 62
one_min.y 39 1111134000 2017-02-02 61
one_min.y 40 1111134600 2017-02-02 60
one_min.y 41 1111135200 2017-02-02 59
one_min.y 42 1111135800 2017-02-02 58
one_min.y 43 1111136400 2017-02-02 57
one_min.y 44 1111137000 2017-02-02 56
one_min.y 45 1111137600 2017-02-02 55
one_min.y 46 1111138200 2017-02-02 54
one_min.y 47 1111138800 2017-02-02 53
one_min.y 48 1111139400 2017-02-02 52
one_min.y 49 1111140000 2017-02-02 51

View File

@ -0,0 +1,84 @@
x?retention=one_min 0 1111110600 2017-02-02 100
x?retention=one_min 3 1111111200 2017-02-02 97
x?retention=one_min 6 1111111800 2017-02-02 94
x?retention=one_min 9 1111112400 2017-02-02 91
x?retention=one_min 12 1111113000 2017-02-02 88
x?retention=one_min 15 1111113600 2017-02-02 85
x?retention=one_min 18 1111114200 2017-02-02 82
x?retention=one_min 21 1111114800 2017-02-02 79
x?retention=one_min 24 1111115400 2017-02-02 76
x?retention=one_min 27 1111116000 2017-02-02 73
x?retention=one_min 30 1111116600 2017-02-02 70
x?retention=one_min 33 1111117200 2017-02-02 67
x?retention=one_min 36 1111117800 2017-02-02 64
x?retention=one_min 39 1111118400 2017-02-02 61
x?retention=one_min 42 1111119000 2017-02-02 58
x?retention=one_min 45 1111119600 2017-02-02 55
x?retention=one_min 48 1111120200 2017-02-02 52
x?retention=one_min 0 1111110600 2017-02-02 100
x?retention=one_min 3 1111111200 2017-02-02 97
x?retention=one_min 6 1111111800 2017-02-02 94
x?retention=one_min 9 1111112400 2017-02-02 91
x?retention=one_min 12 1111113000 2017-02-02 88
x?retention=one_min 15 1111113600 2017-02-02 85
x?retention=one_min 18 1111114200 2017-02-02 82
x?retention=one_min 21 1111114800 2017-02-02 79
x?retention=one_min 24 1111115400 2017-02-02 76
x?retention=one_min 27 1111116000 2017-02-02 73
x?retention=one_min 30 1111116600 2017-02-02 70
x?retention=one_min 33 1111117200 2017-02-02 67
x?retention=one_min 36 1111117800 2017-02-02 64
x?retention=one_min 39 1111118400 2017-02-02 61
x?retention=one_min 42 1111119000 2017-02-02 58
x?retention=one_min 45 1111119600 2017-02-02 55
x?retention=one_min 48 1111120200 2017-02-02 52
y?retention=one_min 0 1111110600 2017-02-02 100
y?retention=one_min 1 1111111200 2017-02-02 99
y?retention=one_min 2 1111111800 2017-02-02 98
y?retention=one_min 3 1111112400 2017-02-02 97
y?retention=one_min 4 1111113000 2017-02-02 96
y?retention=one_min 5 1111113600 2017-02-02 95
y?retention=one_min 6 1111114200 2017-02-02 94
y?retention=one_min 7 1111114800 2017-02-02 93
y?retention=one_min 8 1111115400 2017-02-02 92
y?retention=one_min 9 1111116000 2017-02-02 91
y?retention=one_min 10 1111116600 2017-02-02 90
y?retention=one_min 11 1111117200 2017-02-02 89
y?retention=one_min 12 1111117800 2017-02-02 88
y?retention=one_min 13 1111118400 2017-02-02 87
y?retention=one_min 14 1111119000 2017-02-02 86
y?retention=one_min 15 1111119600 2017-02-02 85
y?retention=one_min 16 1111120200 2017-02-02 84
y?retention=one_min 17 1111120800 2017-02-02 83
y?retention=one_min 18 1111121400 2017-02-02 82
y?retention=one_min 19 1111122000 2017-02-02 81
y?retention=one_min 20 1111122600 2017-02-02 80
y?retention=one_min 21 1111123200 2017-02-02 79
y?retention=one_min 22 1111123800 2017-02-02 78
y?retention=one_min 23 1111124400 2017-02-02 77
y?retention=one_min 24 1111125000 2017-02-02 76
y?retention=one_min 25 1111125600 2017-02-02 75
y?retention=one_min 26 1111126200 2017-02-02 74
y?retention=one_min 27 1111126800 2017-02-02 73
y?retention=one_min 28 1111127400 2017-02-02 72
y?retention=one_min 29 1111128000 2017-02-02 71
y?retention=one_min 30 1111128600 2017-02-02 70
y?retention=one_min 31 1111129200 2017-02-02 69
y?retention=one_min 32 1111129800 2017-02-02 68
y?retention=one_min 33 1111130400 2017-02-02 67
y?retention=one_min 34 1111131000 2017-02-02 66
y?retention=one_min 35 1111131600 2017-02-02 65
y?retention=one_min 36 1111132200 2017-02-02 64
y?retention=one_min 37 1111132800 2017-02-02 63
y?retention=one_min 38 1111133400 2017-02-02 62
y?retention=one_min 39 1111134000 2017-02-02 61
y?retention=one_min 40 1111134600 2017-02-02 60
y?retention=one_min 41 1111135200 2017-02-02 59
y?retention=one_min 42 1111135800 2017-02-02 58
y?retention=one_min 43 1111136400 2017-02-02 57
y?retention=one_min 44 1111137000 2017-02-02 56
y?retention=one_min 45 1111137600 2017-02-02 55
y?retention=one_min 46 1111138200 2017-02-02 54
y?retention=one_min 47 1111138800 2017-02-02 53
y?retention=one_min 48 1111139400 2017-02-02 52
y?retention=one_min 49 1111140000 2017-02-02 51

View File

@ -36,6 +36,8 @@ def test_mutate_and_upgrade(start_cluster):
node1.query("ALTER TABLE mt DELETE WHERE id = 2", settings={"mutations_sync": "2"})
node2.query("SYSTEM SYNC REPLICA mt", timeout=15)
node2.query("DETACH TABLE mt") # stop being leader
node1.query("DETACH TABLE mt") # stop being leader
node1.restart_with_latest_version(signal=9)
node2.restart_with_latest_version(signal=9)
@ -83,6 +85,7 @@ def test_upgrade_while_mutation(start_cluster):
node3.query("SYSTEM STOP MERGES mt1")
node3.query("ALTER TABLE mt1 DELETE WHERE id % 2 == 0")
node3.query("DETACH TABLE mt1") # stop being leader
node3.restart_with_latest_version(signal=9)
# checks for readonly

View File

@ -1 +1,14 @@
-0
nan
nan
1
2
3
4
5
6
7
8
9
nan
nan

View File

@ -1 +1,5 @@
SELECT round(avgWeighted(x, y)) FROM (SELECT 1023 AS x, 1000000000 AS y UNION ALL SELECT 10 AS x, -9223372036854775808 AS y);
select avgWeighted(number, toDecimal128(number, 9)) from numbers(0);
SELECT avgWeighted(a, toDecimal64(c, 9)) OVER (PARTITION BY c) FROM (SELECT number AS a, number AS c FROM numbers(10));
select avg(toDecimal128(number, 9)) from numbers(0);
select avgWeighted(number, toDecimal128(0, 9)) from numbers(10);

View File

@ -21,7 +21,7 @@ CREATE TABLE system.events\n(\n `event` String,\n `value` UInt64,\n `de
CREATE TABLE system.formats\n(\n `name` String,\n `is_input` UInt8,\n `is_output` UInt8\n)\nENGINE = SystemFormats()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'
CREATE TABLE system.functions\n(\n `name` String,\n `is_aggregate` UInt8,\n `case_insensitive` UInt8,\n `alias_to` String,\n `create_query` String,\n `origin` Enum8(\'System\' = 0, \'SQLUserDefined\' = 1, \'ExecutableUserDefined\' = 2)\n)\nENGINE = SystemFunctions()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'
CREATE TABLE system.grants\n(\n `user_name` Nullable(String),\n `role_name` Nullable(String),\n `access_type` Enum8(\'SQLITE\' = -128, \'ODBC\' = -127, \'JDBC\' = -126, \'HDFS\' = -125, \'S3\' = -124, \'SOURCES\' = -123, \'ALL\' = -122, \'NONE\' = -121, \'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'MOVE PARTITION BETWEEN SHARDS\' = 62, \'CREATE USER\' = 63, \'ALTER USER\' = 64, \'DROP USER\' = 65, \'CREATE ROLE\' = 66, \'ALTER ROLE\' = 67, \'DROP ROLE\' = 68, \'ROLE ADMIN\' = 69, \'CREATE ROW POLICY\' = 70, \'ALTER ROW POLICY\' = 71, \'DROP ROW POLICY\' = 72, \'CREATE QUOTA\' = 73, \'ALTER QUOTA\' = 74, \'DROP QUOTA\' = 75, \'CREATE SETTINGS PROFILE\' = 76, \'ALTER SETTINGS PROFILE\' = 77, \'DROP SETTINGS PROFILE\' = 78, \'SHOW USERS\' = 79, \'SHOW ROLES\' = 80, \'SHOW ROW POLICIES\' = 81, \'SHOW QUOTAS\' = 82, \'SHOW SETTINGS PROFILES\' = 83, \'SHOW ACCESS\' = 84, \'ACCESS MANAGEMENT\' = 85, \'SYSTEM SHUTDOWN\' = 86, \'SYSTEM DROP DNS CACHE\' = 87, \'SYSTEM DROP MARK CACHE\' = 88, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 89, \'SYSTEM DROP MMAP CACHE\' = 90, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 91, \'SYSTEM DROP CACHE\' = 92, \'SYSTEM RELOAD CONFIG\' = 93, \'SYSTEM RELOAD SYMBOLS\' = 94, \'SYSTEM RELOAD DICTIONARY\' = 95, \'SYSTEM RELOAD MODEL\' = 96, \'SYSTEM RELOAD FUNCTION\' = 97, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 98, \'SYSTEM RELOAD\' = 99, \'SYSTEM RESTART DISK\' = 100, \'SYSTEM MERGES\' = 101, \'SYSTEM TTL MERGES\' = 102, \'SYSTEM FETCHES\' = 103, \'SYSTEM MOVES\' = 104, \'SYSTEM DISTRIBUTED SENDS\' = 105, \'SYSTEM REPLICATED SENDS\' = 106, \'SYSTEM SENDS\' = 107, \'SYSTEM REPLICATION QUEUES\' = 108, \'SYSTEM DROP REPLICA\' = 109, \'SYSTEM SYNC REPLICA\' = 110, \'SYSTEM RESTART REPLICA\' = 111, \'SYSTEM RESTORE REPLICA\' = 112, \'SYSTEM FLUSH DISTRIBUTED\' = 113, \'SYSTEM FLUSH LOGS\' = 114, \'SYSTEM FLUSH\' = 115, \'SYSTEM\' = 116, \'dictGet\' = 117, \'addressToLine\' = 118, \'addressToSymbol\' = 119, \'demangle\' = 120, \'INTROSPECTION\' = 121, \'FILE\' = 122, \'URL\' = 123, \'REMOTE\' = 124, \'MONGO\' = 125, \'MYSQL\' = 126, \'POSTGRES\' = 127),\n `database` Nullable(String),\n `table` Nullable(String),\n `column` Nullable(String),\n `is_partial_revoke` UInt8,\n `grant_option` UInt8\n)\nENGINE = SystemGrants()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'
CREATE TABLE system.graphite_retentions\n(\n `config_name` String,\n `regexp` String,\n `function` String,\n `age` UInt64,\n `precision` UInt64,\n `priority` UInt16,\n `is_default` UInt8,\n `Tables.database` Array(String),\n `Tables.table` Array(String)\n)\nENGINE = SystemGraphite()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'
CREATE TABLE system.graphite_retentions\n(\n `config_name` String,\n `rule_type` String,\n `regexp` String,\n `function` String,\n `age` UInt64,\n `precision` UInt64,\n `priority` UInt16,\n `is_default` UInt8,\n `Tables.database` Array(String),\n `Tables.table` Array(String)\n)\nENGINE = SystemGraphite()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'
CREATE TABLE system.licenses\n(\n `library_name` String,\n `license_type` String,\n `license_path` String,\n `license_text` String\n)\nENGINE = SystemLicenses()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'
CREATE TABLE system.macros\n(\n `macro` String,\n `substitution` String\n)\nENGINE = SystemMacros()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'
CREATE TABLE system.merge_tree_settings\n(\n `name` String,\n `value` String,\n `changed` UInt8,\n `description` String,\n `type` String\n)\nENGINE = SystemMergeTreeSettings()\nCOMMENT \'SYSTEM TABLE is built on the fly.\'

View File

@ -1,6 +0,0 @@
0 0
1 1
20000
0
0 20000
1 20001

View File

@ -1,49 +0,0 @@
#!/usr/bin/env bash
# Tags: long
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q "create table many_mutations (x UInt32, y UInt32) engine = MergeTree order by x"
$CLICKHOUSE_CLIENT -q "insert into many_mutations values (0, 0), (1, 1)"
$CLICKHOUSE_CLIENT -q "system stop merges many_mutations"
$CLICKHOUSE_CLIENT -q "select x, y from many_mutations order by x"
job()
{
for i in {1..1000}
do
$CLICKHOUSE_CLIENT -q "alter table many_mutations update y = y + 1 where 1"
done
}
job &
job &
job &
job &
job &
job &
job &
job &
job &
job &
job &
job &
job &
job &
job &
job &
job &
job &
job &
job &
wait
$CLICKHOUSE_CLIENT -q "select count() from system.mutations where database = currentDatabase() and table = 'many_mutations' and not is_done"
$CLICKHOUSE_CLIENT -q "system start merges many_mutations"
$CLICKHOUSE_CLIENT -q "optimize table many_mutations final"
$CLICKHOUSE_CLIENT -q "select count() from system.mutations where database = currentDatabase() and table = 'many_mutations' and not is_done"
$CLICKHOUSE_CLIENT -q "select x, y from many_mutations order by x"

View File

@ -0,0 +1 @@
select count(1) from (SELECT 1 AS a, count(1) FROM numbers(5))

View File

@ -32,6 +32,7 @@ if (NOT DEFINED ENABLE_UTILS OR ENABLE_UTILS)
add_subdirectory (wal-dump)
add_subdirectory (check-mysql-binlog)
add_subdirectory (keeper-bench)
add_subdirectory (graphite-rollup)
if (USE_NURAFT)
add_subdirectory (keeper-data-dumper)

View File

@ -74,7 +74,7 @@ class Backport:
# First pass. Find all must-backports
for label in pr['labels']['nodes']:
if label['name'] == 'pr-bugfix' or label['name'] == 'pr-must-backport':
if label['name'] == 'pr-must-backport':
backport_map[pr['number']] = branch_set.copy()
continue
matched = RE_MUST_BACKPORT.match(label['name'])

View File

@ -0,0 +1,23 @@
add_executable(graphite-rollup-bench graphite-rollup-bench.cpp)
target_link_libraries(
graphite-rollup-bench
PRIVATE
clickhouse_storages_system
clickhouse_aggregate_functions
clickhouse_common_config
dbms
)
target_include_directories(
graphite-rollup-bench
PRIVATE
${ClickHouse_SOURCE_DIR}/src ${CMAKE_BINARY_DIR}/src
${ClickHouse_SOURCE_DIR}/base ${ClickHouse_SOURCE_DIR}/base/pcg-random
${CMAKE_BINARY_DIR}/src/Core/include
${POCO_INCLUDE_DIR}
${ClickHouse_SOURCE_DIR}/contrib/double-conversion ${ClickHouse_SOURCE_DIR}/contrib/dragonbox/include
${ClickHouse_SOURCE_DIR}/contrib/fmtlib/include
${ClickHouse_SOURCE_DIR}/contrib/cityhash102/include
${RE2_INCLUDE_DIR} ${CMAKE_BINARY_DIR}/contrib/re2_st
)
target_compile_definitions(graphite-rollup-bench PRIVATE RULES_DIR="${CMAKE_CURRENT_SOURCE_DIR}")

View File

@ -0,0 +1,147 @@
#include <chrono>
#include <cstdio>
#include <cstdlib>
#include <iomanip>
#include <iostream>
#include <stdexcept>
#include <system_error>
#include <boost/program_options.hpp>
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <Processors/Merges/Algorithms/Graphite.h>
#include <Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h>
#include <Storages/System/StorageSystemGraphite.h>
#include <Common/Config/ConfigProcessor.h>
#include <Common/Exception.h>
#include <Interpreters/Context.h>
using namespace DB;
static SharedContextHolder shared_context = Context::createShared();
std::vector<StringRef> loadMetrics(const std::string & metrics_file)
{
std::vector<StringRef> metrics;
FILE * stream;
char * line = nullptr;
size_t len = 0;
ssize_t nread;
stream = fopen(metrics_file.c_str(), "r");
if (stream == nullptr)
{
throw std::runtime_error(strerror(errno));
}
while ((nread = getline(&line, &len, stream)) != -1)
{
size_t l = strlen(line);
if (l > 0)
{
if (line[l - 1] == '\n')
{
line[l - 1] = '\0';
l--;
}
if (l > 0)
{
metrics.push_back(StringRef(strdup(line), l));
}
}
}
free(line);
if (ferror(stream))
{
fclose(stream);
throw std::runtime_error(strerror(errno));
}
fclose(stream);
return metrics;
}
ConfigProcessor::LoadedConfig loadConfiguration(const std::string & config_path)
{
ConfigProcessor config_processor(config_path, true, true);
ConfigProcessor::LoadedConfig config = config_processor.loadConfig(false);
return config;
}
void bench(const std::string & config_path, const std::string & metrics_file, size_t n, bool verbose)
{
auto config = loadConfiguration(config_path);
auto context = Context::createGlobal(shared_context.get());
context->setConfig(config.configuration.get());
Graphite::Params params;
setGraphitePatternsFromConfig(context, "graphite_rollup", params);
std::vector<StringRef> metrics = loadMetrics(metrics_file);
std::vector<double> durations(metrics.size());
size_t j, i;
for (j = 0; j < n; j++)
{
for (i = 0; i < metrics.size(); i++)
{
auto start = std::chrono::high_resolution_clock::now();
auto rule = DB::Graphite::selectPatternForPath(params, metrics[i]);
(void)rule;
auto end = std::chrono::high_resolution_clock::now();
double duration = (duration_cast<std::chrono::duration<double>>(end - start)).count() * 1E9;
durations[i] += duration;
if (j == 0 && verbose)
{
std::cout << metrics[i].data << ": rule with regexp '" << rule.second->regexp_str << "' found\n";
}
}
}
for (i = 0; i < metrics.size(); i++)
{
std::cout << metrics[i].data << " " << durations[i] / n << " ns\n";
free(const_cast<void *>(static_cast<const void *>(metrics[i].data)));
}
}
int main(int argc, char ** argv)
{
registerAggregateFunctions();
std::string config_file, metrics_file;
using namespace std::literals;
std::string config_default = RULES_DIR + "/rollup.xml"s;
std::string metrics_default = RULES_DIR + "/metrics.txt"s;
namespace po = boost::program_options;
po::variables_map vm;
po::options_description desc;
desc.add_options()("help,h", "produce help")(
"config,c", po::value<std::string>()->default_value(config_default), "XML config with rollup rules")(
"metrics,m", po::value<std::string>()->default_value(metrics_default), "metrcis files (one metric per line) for run benchmark")(
"verbose,V", po::bool_switch()->default_value(false), "verbose output (print found rule)");
po::parsed_options parsed = po::command_line_parser(argc, argv).options(desc).run();
po::store(parsed, vm);
po::notify(vm);
if (vm.count("help"))
{
std::cout << desc << '\n';
exit(1);
}
bench(vm["config"].as<std::string>(), vm["metrics"].as<std::string>(), 10000, vm["verbose"].as<bool>());
return 0;
}

View File

@ -0,0 +1,11 @@
test.sum
sum?env=test&tag=Fake3
test.max
max?env=test&tag=Fake4
test.min
min?env=test&tag=Fake5
fake5?env=test&tag=Fake5
test.p95
p95?env=test&tag=FakeNo
default
default?env=test&tag=FakeNo

View File

@ -0,0 +1,167 @@
<yandex>
<graphite_rollup>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.sum$</regexp>
<function>sum</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<rule_type>tagged</rule_type>
<regexp>^((.*)|.)sum\?</regexp>
<function>sum</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.max$</regexp>
<function>max</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<rule_type>tagged</rule_type>
<regexp>^((.*)|.)max\?</regexp>
<function>max</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.min$</regexp>
<function>min</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<rule_type>tagged</rule_type>
<regexp>^((.*)|.)min\?</regexp>
<function>min</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.fake1\..*\.Fake1\.</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>tag_list</rule_type>
<regexp>fake1;tag=Fake1</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.fake2\..*\.Fake2\.</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>tag_list</rule_type>
<regexp>fake2;tag=Fake2</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.fake3\..*\.Fake3\.</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>tag_list</rule_type>
<regexp>fake3;tag=Fake3</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.fake4\..*\.Fake4\.</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>tag_list</rule_type>
<regexp>fake4;tag=Fake4</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.fake5\..*\.Fake5\.</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>tag_list</rule_type>
<regexp>fake5;tag=Fake5</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.fake6\..*\.Fake6\.</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>tag_list</rule_type>
<regexp>fake6;tag=Fake6</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.fake7\..*\.Fake7\.</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>tag_list</rule_type>
<regexp>fake7;tag=Fake7</regexp>
<function>sum</function>
</pattern>
<default>
<function>avg</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>3600</age>
<precision>300</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</default>
</graphite_rollup>
</yandex>

View File

@ -0,0 +1,167 @@
<yandex>
<graphite_rollup>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.sum$</regexp>
<function>sum</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<rule_type>tagged</rule_type>
<regexp>^((.*)|.)sum\?</regexp>
<function>sum</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.max$</regexp>
<function>max</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<rule_type>tagged</rule_type>
<regexp>^((.*)|.)max\?</regexp>
<function>max</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.min$</regexp>
<function>min</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<rule_type>tagged</rule_type>
<regexp>^((.*)|.)min\?</regexp>
<function>min</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.fake1\..*\.Fake1\.</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>tagged</rule_type>
<regexp><![CDATA[^fake1\?(.*&)*tag=Fake1(&|$)]]></regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.fake2\..*\.Fake2\.</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>tagged</rule_type>
<regexp><![CDATA[^fake2\?(.*&)*tag=Fake2(&|$)]]></regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.fake3\..*\.Fake3\.</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>tagged</rule_type>
<regexp><![CDATA[^fake3\?(.*&)*tag=Fake3(&|$)]]></regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.fake4\..*\.Fake4\.</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>tagged</rule_type>
<regexp><![CDATA[^fake4\?(.*&)*tag=Fake4(&|$)]]></regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.fake5\..*\.Fake5\.</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>tagged</rule_type>
<regexp><![CDATA[^fake5\?(.*&)*tag=Fake5(&|$)]]></regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.fake6\..*\.Fake6\.</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>tagged</rule_type>
<regexp><![CDATA[^fake6\?(.*&)*tag=Fake6(&|$)]]></regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>plain</rule_type>
<regexp>\.fake7\..*\.Fake7\.</regexp>
<function>sum</function>
</pattern>
<pattern>
<rule_type>tagged</rule_type>
<regexp><![CDATA[^fake7\?(.*&)*tag=Fake7(&|$)]]></regexp>
<function>sum</function>
</pattern>
<default>
<function>avg</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>3600</age>
<precision>300</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</default>
</graphite_rollup>
</yandex>

View File

@ -0,0 +1,147 @@
<yandex>
<graphite_rollup>
<pattern>
<regexp>\.sum$</regexp>
<function>sum</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<regexp>^((.*)|.)sum\?</regexp>
<function>sum</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<regexp>\.max$</regexp>
<function>max</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<regexp>^((.*)|.)max\?</regexp>
<function>max</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<regexp>\.min$</regexp>
<function>min</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<regexp>^((.*)|.)min\?</regexp>
<function>min</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</pattern>
<pattern>
<regexp>\.fake1\..*\.Fake1\.</regexp>
<function>sum</function>
</pattern>
<pattern>
<regexp><![CDATA[^fake1\?(.*&)*tag=Fake1(&|$)]]></regexp>
<function>sum</function>
</pattern>
<pattern>
<regexp>\.fake2\..*\.Fake2\.</regexp>
<function>sum</function>
</pattern>
<pattern>
<regexp><![CDATA[^fake2\?(.*&)*tag=Fake2(&|$)]]></regexp>
<function>sum</function>
</pattern>
<pattern>
<regexp>\.fake3\..*\.Fake3\.</regexp>
<function>sum</function>
</pattern>
<pattern>
<regexp><![CDATA[^fake3\?(.*&)*tag=Fake3(&|$)]]></regexp>
<function>sum</function>
</pattern>
<pattern>
<regexp>\.fake4\..*\.Fake4\.</regexp>
<function>sum</function>
</pattern>
<pattern>
<regexp><![CDATA[^fake4\?(.*&)*tag=Fake4(&|$)]]></regexp>
<function>sum</function>
</pattern>
<pattern>
<regexp>\.fake5\..*\.Fake5\.</regexp>
<function>sum</function>
</pattern>
<pattern>
<regexp><![CDATA[^fake5\?(.*&)*tag=Fake5(&|$)]]></regexp>
<function>sum</function>
</pattern>
<pattern>
<regexp>\.fake6\..*\.Fake6\.</regexp>
<function>sum</function>
</pattern>
<pattern>
<regexp><![CDATA[^fake6\?(.*&)*tag=Fake6(&|$)]]></regexp>
<function>sum</function>
</pattern>
<pattern>
<regexp>\.fake7\..*\.Fake7\.</regexp>
<function>sum</function>
</pattern>
<pattern>
<regexp><![CDATA[^fake7\?(.*&)*tag=Fake7(&|$)]]></regexp>
<function>sum</function>
</pattern>
<default>
<function>avg</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>3600</age>
<precision>300</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</default>
</graphite_rollup>
</yandex>