Merge branch 'master' into consistent_metadata2

This commit is contained in:
alesapin 2020-05-25 20:23:52 +03:00
commit d7c59bb4aa
24 changed files with 259 additions and 72 deletions

View File

@ -1265,4 +1265,63 @@ Possible values:
Default value: 16.
## low_cardinality_max_dictionary_size {#low_cardinality_max_dictionary_size}
Sets a maximum size in rows of a shared global dictionary for the [LowCardinality](../../sql-reference/data-types/lowcardinality.md) data type that can be written to a storage file system. This setting prevents issues with RAM in case of unlimited dictionary growth. All the data that can't be encoded due to maximum dictionary size limitation ClickHouse writes in an ordinary method.
Possible values:
- Any positive integer.
Default value: 8192.
## low_cardinality_use_single_dictionary_for_part {#low_cardinality_use_single_dictionary_for_part}
Turns on or turns off using of single dictionary for the data part.
By default, ClickHouse server monitors the size of dictionaries and if a dictionary overflows then the server starts to write the next one. To prohibit creating several dictionaries set `low_cardinality_use_single_dictionary_for_part = 1`.
Possible values:
- 1 — Creating several dictionaries for the data part is prohibited.
- 0 — Creating several dictionaries for the data part is not prohibited.
Default value: 0.
## low_cardinality_allow_in_native_format {#low_cardinality_allow_in_native_format}
Allows or restricts using the [LowCardinality](../../sql-reference/data-types/lowcardinality.md) data type with the [Native](../../interfaces/formats.md#native) format.
If usage of `LowCardinality` is restricted, ClickHouse server converts `LowCardinality`-columns to ordinary ones for `SELECT` queries, and convert ordinary columns to `LowCardinality`-columns for `INSERT` queries.
This setting is required mainly for third-party clients which don't support `LowCardinality` data type.
Possible values:
- 1 — Usage of `LowCardinality` is not restricted.
- 0 — Usage of `LowCardinality` is restricted.
Default value: 1.
## allow_suspicious_low_cardinality_types {#allow_suspicious_low_cardinality_types}
Allows or restricts using [LowCardinality](../../sql-reference/data-types/lowcardinality.md) with data types with fixed size of 8 bytes or less: numeric data types and `FixedString(8_bytes_or_less)`.
For small fixed values using of `LowCardinality` is usually inefficient, because ClickHouse stores a numeric index for each row. As a result:
- Disk space usage can rise.
- RAM consumption can be higher, depending on a dictionary size.
- Some functions can work slower due to extra coding/encoding operations.
Merge times in [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md)-engine tables can grow due to all the reasons described above.
Possible values:
- 1 — Usage of `LowCardinality` is not restricted.
- 0 — Usage of `LowCardinality` is restricted.
Default value: 0.
[Original article](https://clickhouse.tech/docs/en/operations/settings/settings/) <!-- hide -->

View File

@ -1,5 +1,5 @@
---
toc_priority: 52
toc_priority: 53
toc_title: AggregateFunction
---

View File

@ -1,5 +1,5 @@
---
toc_priority: 51
toc_priority: 52
toc_title: Array(T)
---

View File

@ -0,0 +1,59 @@
---
toc_priority: 51
toc_title: LowCardinality
---
# LowCardinality Data Type {#lowcardinality-data-type}
Changes the internal representation of other data types to be dictionary-encoded.
## Syntax {#lowcardinality-syntax}
```sql
LowCardinality(data_type)
```
**Parameters**
- `data_type` — [String](string.md), [FixedString](fixedstring.md), [Date](date.md), [DateTime](datetime.md), and numbers excepting [Decimal](decimal.md). `LowCardinality` is not efficient for some data types, see the [allow_suspicious_low_cardinality_types](../../operations/settings/settings.md#allow_suspicious_low_cardinality_types) setting description.
## Description {#lowcardinality-dscr}
`LowCardinality` is a superstructure that changes a data storage method and rules of data processing. ClickHouse applies [dictionary coding](https://en.wikipedia.org/wiki/Dictionary_coder) to `LowCardinality`-columns. Operating with dictionary encoded data significantly increases performance of [SELECT](../statements/select/index.md) queries for many applications.
The efficiency of using `LowCarditality` data type depends on data diversity. If a dictionary contains less than 10,000 distinct values, then ClickHouse mostly shows higher efficiency of data reading and storing. If a dictionary contains more than 100,000 distinct values, then ClickHouse can perform worse in comparison with using ordinary data types.
Consider using `LowCardinality` instead of [Enum](enum.md) when working with strings. `LowCardinality` provides more flexibility in use and often reveals the same or higher efficiency.
## Example
Create a table with a `LowCardinality`-column:
```sql
CREATE TABLE lc_t
(
`id` UInt16,
`strings` LowCardinality(String)
)
ENGINE = MergeTree()
ORDER BY id
```
## Related Settings and Functions
Settings:
- [low_cardinality_max_dictionary_size](../../operations/settings/settings.md#low_cardinality_max_dictionary_size)
- [low_cardinality_use_single_dictionary_for_part](../../operations/settings/settings.md#low_cardinality_use_single_dictionary_for_part)
- [low_cardinality_allow_in_native_format](../../operations/settings/settings.md#low_cardinality_allow_in_native_format)
- [allow_suspicious_low_cardinality_types](../../operations/settings/settings.md#allow_suspicious_low_cardinality_types)
Functions:
- [toLowCardinality](../functions/type-conversion-functions.md#tolowcardinality)
## See Also
- [A Magical Mystery Tour of the LowCardinality Data Type](https://www.altinity.com/blog/2019/3/27/low-cardinality).
- [Reducing Clickhouse Storage Cost with the Low Cardinality Type Lessons from an Instana Engineer](https://www.instana.com/blog/reducing-clickhouse-storage-cost-with-the-low-cardinality-type-lessons-from-an-instana-engineer/).
- [String Optimization (video presentation in Russian)](https://youtu.be/rqf-ILRgBdY?list=PL0Z2YDlm0b3iwXCpEFiOOYmwXzVmjJfEt). [Slides in English](https://github.com/yandex/clickhouse-presentations/raw/master/meetup19/string_optimization.pdf).

View File

@ -1,5 +1,5 @@
---
toc_priority: 54
toc_priority: 55
toc_title: Nullable
---

View File

@ -1,5 +1,5 @@
---
toc_priority: 53
toc_priority: 54
toc_title: Tuple(T1, T2, ...)
---

View File

@ -516,7 +516,7 @@ Result:
**See Also**
- \[ISO 8601 announcement by @xkcd\](https://xkcd.com/1179/)
- [ISO 8601 announcement by @xkcd](https://xkcd.com/1179/)
- [RFC 1123](https://tools.ietf.org/html/rfc1123)
- [toDate](#todate)
- [toDateTime](#todatetime)
@ -529,4 +529,43 @@ Same as for [parseDateTimeBestEffort](#parsedatetimebesteffort) except that it r
Same as for [parseDateTimeBestEffort](#parsedatetimebesteffort) except that it returns zero date or zero date time when it encounters a date format that cannot be processed.
## toLowCardinality {#tolowcardinality}
Converts input parameter to the [LowCardianlity](../data-types/lowcardinality.md) version of same data type.
To convert data from the `LowCardinality` data type use the [CAST](#type_conversion_function-cast) function. For example, `CAST(x as String)`.
**Syntax**
```sql
toLowCardinality(expr)
```
**Parameters**
- `expr` — [Expression](../syntax.md#syntax-expressions) resulting in one of the [supported data types](../data-types/index.md#data_types).
**Returned values**
- Result of `expr`.
Type: `LowCardinality(expr_result_type)`
**Example**
Query:
```sql
SELECT toLowCardinality('1')
```
Result:
```text
┌─toLowCardinality('1')─┐
│ 1 │
└───────────────────────┘
```
[Original article](https://clickhouse.tech/docs/en/query_language/functions/type_conversion_functions/) <!--hide-->

View File

@ -1533,7 +1533,7 @@ void InterpreterSelectQuery::executeFetchColumns(
if constexpr (pipeline_with_processors)
{
if (streams.size() == 1 || pipes.size() == 1)
pipeline.setMaxThreads(streams.size());
pipeline.setMaxThreads(1);
/// Unify streams. They must have same headers.
if (streams.size() > 1)

View File

@ -271,6 +271,11 @@ QueryPipeline InterpreterSelectWithUnionQuery::executeWithProcessors()
{
auto common_header = getCommonHeaderForUnion(headers);
main_pipeline.unitePipelines(std::move(pipelines), common_header);
// nested queries can force 1 thread (due to simplicity)
// but in case of union this cannot be done.
UInt64 max_threads = context->getSettingsRef().max_threads;
main_pipeline.setMaxThreads(std::min<UInt64>(nested_interpreters.size(), max_threads));
}
main_pipeline.addInterpreterContext(context);

View File

@ -234,14 +234,6 @@ def run_tests_array(all_tests_with_params):
clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE)
clickhouse_proc.communicate("SELECT 'Running test {suite}/{case} from pid={pid}';".format(pid = os.getpid(), case = case, suite = suite))
if not args.no_system_log_cleanup:
clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE)
clickhouse_proc.communicate("SYSTEM FLUSH LOGS")
for table in ['query_log', 'query_thread_log', 'trace_log', 'metric_log']:
clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE)
clickhouse_proc.communicate("TRUNCATE TABLE IF EXISTS system.{}".format(table))
reference_file = os.path.join(suite_dir, name) + '.reference'
stdout_file = os.path.join(suite_tmp_dir, name) + '.stdout'
stderr_file = os.path.join(suite_tmp_dir, name) + '.stderr'
@ -572,7 +564,6 @@ if __name__ == '__main__':
parser.add_argument('--stop', action='store_true', default=None, dest='stop', help='Stop on network errors')
parser.add_argument('--order', default='desc', choices=['asc', 'desc', 'random'], help='Run order')
parser.add_argument('--testname', action='store_true', default=None, dest='testname', help='Make query with test name before test run')
parser.add_argument('--no-system-log-cleanup', action='store_true', default=None, help='Do not cleanup system.*_log tables')
parser.add_argument('--hung-check', action='store_true', default=False)
parser.add_argument('--force-color', action='store_true', default=False)
parser.add_argument('--database', help='Database for tests (random name test_XXXXXX by default)')

View File

@ -47,7 +47,7 @@ SELECT
threads_realtime >= threads_time_user_system_io,
any(length(thread_ids)) >= 1
FROM
(SELECT * FROM system.query_log PREWHERE query='$heavy_cpu_query' WHERE type='QueryFinish' ORDER BY event_time DESC LIMIT 1)
(SELECT * FROM system.query_log PREWHERE query='$heavy_cpu_query' WHERE event_date >= today()-1 AND type=2 ORDER BY event_time DESC LIMIT 1)
ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV"
# Check per-thread and per-query ProfileEvents consistency
@ -58,7 +58,7 @@ SELECT PN, PVq, PVt FROM
SELECT PN, sum(PV) AS PVt
FROM system.query_thread_log
ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV
WHERE query_id='$query_id'
WHERE event_date >= today()-1 AND query_id='$query_id'
GROUP BY PN
) js1
ANY INNER JOIN
@ -66,7 +66,7 @@ ANY INNER JOIN
SELECT PN, PV AS PVq
FROM system.query_log
ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV
WHERE query_id='$query_id'
WHERE event_date >= today()-1 AND query_id='$query_id'
) js2
USING PN
WHERE

View File

@ -19,7 +19,7 @@ $CLICKHOUSE_CLIENT --use_uncompressed_cache=1 --query_id="test-query-uncompresse
sleep 1
$CLICKHOUSE_CLIENT --query="SYSTEM FLUSH LOGS"
$CLICKHOUSE_CLIENT --query="SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'Seek')], ProfileEvents.Values[indexOf(ProfileEvents.Names, 'ReadCompressedBytes')], ProfileEvents.Values[indexOf(ProfileEvents.Names, 'UncompressedCacheHits')] AS hit FROM system.query_log WHERE (query_id = 'test-query-uncompressed-cache') AND (type = 'QueryFinish') ORDER BY event_time DESC LIMIT 1"
$CLICKHOUSE_CLIENT --query="SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'Seek')], ProfileEvents.Values[indexOf(ProfileEvents.Names, 'ReadCompressedBytes')], ProfileEvents.Values[indexOf(ProfileEvents.Names, 'UncompressedCacheHits')] AS hit FROM system.query_log WHERE (query_id = 'test-query-uncompressed-cache') AND (type = 2) AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1"
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS small_table"

View File

@ -95,7 +95,7 @@ echo 7
# and finally querylog
$CLICKHOUSE_CLIENT \
--server_logs_file=/dev/null \
--query="select * from system.query_log where query like '%TOPSECRET%';"
--query="select * from system.query_log where event_time>now() - 10 and query like '%TOPSECRET%';"
rm -f $tmp_file >/dev/null 2>&1
@ -117,8 +117,8 @@ sleep 0.1;
echo 9
$CLICKHOUSE_CLIENT \
--server_logs_file=/dev/null \
--query="SELECT if( count() > 0, 'text_log non empty', 'text_log empty') FROM system.text_log WHERE message like '%find_me%';
select * from system.text_log where message like '%TOPSECRET=TOPSECRET%';" --ignore-error --multiquery
--query="SELECT if( count() > 0, 'text_log non empty', 'text_log empty') FROM system.text_log WHERE event_time>now() - 60 and message like '%find_me%';
select * from system.text_log where event_time>now() - 60 and message like '%TOPSECRET=TOPSECRET%';" --ignore-error --multiquery
echo 'finish'
rm -f $tmp_file >/dev/null 2>&1

View File

@ -10,7 +10,7 @@ do
${CLICKHOUSE_CLIENT} --query="SYSTEM FLUSH LOGS"
sleep 0.1;
if [[ $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT count() > 0 FROM system.text_log WHERE position(system.text_log.message, 'SELECT 6103') > 0") == 1 ]]; then echo 1; exit; fi;
if [[ $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT count() > 0 FROM system.text_log WHERE position(system.text_log.message, 'SELECT 6103') > 0 AND event_date >= yesterday()") == 1 ]]; then echo 1; exit; fi;
done;

View File

@ -3,5 +3,5 @@ SELECT * FROM test_table_for_01070_exception_code_in_query_log_table; -- { serve
CREATE TABLE test_table_for_01070_exception_code_in_query_log_table (value UInt64) ENGINE=Memory();
SELECT * FROM test_table_for_01070_exception_code_in_query_log_table;
SYSTEM FLUSH LOGS;
SELECT exception_code FROM system.query_log WHERE query = 'SELECT * FROM test_table_for_01070_exception_code_in_query_log_table' ORDER BY exception_code;
SELECT exception_code FROM system.query_log WHERE query = 'SELECT * FROM test_table_for_01070_exception_code_in_query_log_table' AND event_date >= yesterday() AND event_time > now() - INTERVAL 5 MINUTE ORDER BY exception_code;
DROP TABLE IF EXISTS test_table_for_01070_exception_code_in_query_log_table;

View File

@ -8,13 +8,13 @@ WITH
(
SELECT query_id
FROM system.query_log
WHERE (query = 'SELECT 1')
WHERE (query = 'SELECT 1') AND (event_date >= (today() - 1))
ORDER BY event_time DESC
LIMIT 1
) AS id
SELECT uniqExact(thread_id)
FROM system.query_thread_log
WHERE (query_id = id) AND (thread_id != master_thread_id);
WHERE (event_date >= (today() - 1)) AND (query_id = id) AND (thread_id != master_thread_id);
select sum(number) from numbers(1000000);
SYSTEM FLUSH LOGS;
@ -23,13 +23,13 @@ WITH
(
SELECT query_id
FROM system.query_log
WHERE (query = 'SELECT sum(number) FROM numbers(1000000)')
WHERE (query = 'SELECT sum(number) FROM numbers(1000000)') AND (event_date >= (today() - 1))
ORDER BY event_time DESC
LIMIT 1
) AS id
SELECT uniqExact(thread_id)
FROM system.query_thread_log
WHERE (query_id = id) AND (thread_id != master_thread_id);
WHERE (event_date >= (today() - 1)) AND (query_id = id) AND (thread_id != master_thread_id);
select sum(number) from numbers_mt(1000000);
SYSTEM FLUSH LOGS;
@ -38,10 +38,10 @@ WITH
(
SELECT query_id
FROM system.query_log
WHERE (query = 'SELECT sum(number) FROM numbers_mt(1000000)')
WHERE (query = 'SELECT sum(number) FROM numbers_mt(1000000)') AND (event_date >= (today() - 1))
ORDER BY event_time DESC
LIMIT 1
) AS id
SELECT uniqExact(thread_id) > 2
FROM system.query_thread_log
WHERE (query_id = id) AND (thread_id != master_thread_id);
WHERE (event_date >= (today() - 1)) AND (query_id = id) AND (thread_id != master_thread_id);

View File

@ -3,4 +3,4 @@ SET allow_introspection_functions = 1;
SET memory_profiler_step = 1000000;
SELECT ignore(groupArray(number), 'test memory profiler') FROM numbers(10000000);
SYSTEM FLUSH LOGS;
WITH addressToSymbol(arrayJoin(trace)) AS symbol SELECT count() > 0 FROM system.trace_log t WHERE trace_type = 'Memory' AND query_id = (SELECT query_id FROM system.query_log WHERE query LIKE '%test memory profiler%' ORDER BY event_time DESC LIMIT 1);
WITH addressToSymbol(arrayJoin(trace)) AS symbol SELECT count() > 0 FROM system.trace_log t WHERE event_date >= yesterday() AND trace_type = 'Memory' AND query_id = (SELECT query_id FROM system.query_log WHERE event_date >= yesterday() AND query LIKE '%test memory profiler%' ORDER BY event_time DESC LIMIT 1);

View File

@ -3,4 +3,4 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT --quota_key Hello --query_id test_quota_key --log_queries 1 --multiquery --query "SELECT 1; SYSTEM FLUSH LOGS; SELECT DISTINCT quota_key FROM system.query_log WHERE query_id = 'test_quota_key'"
$CLICKHOUSE_CLIENT --quota_key Hello --query_id test_quota_key --log_queries 1 --multiquery --query "SELECT 1; SYSTEM FLUSH LOGS; SELECT DISTINCT quota_key FROM system.query_log WHERE event_date >= yesterday() AND event_time >= now() - 300 AND query_id = 'test_quota_key'"

View File

@ -2,14 +2,14 @@ set log_queries=1;
select '01231_log_queries_min_type/QUERY_START';
system flush logs;
select count() from system.query_log where query like '%01231_log_queries_min_type/%' and query not like '%system.query_log%';
select count() from system.query_log where query like '%01231_log_queries_min_type/%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute;
set log_queries_min_type='EXCEPTION_BEFORE_START';
select '01231_log_queries_min_type/EXCEPTION_BEFORE_START';
system flush logs;
select count() from system.query_log where query like '%01231_log_queries_min_type/%' and query not like '%system.query_log%';
select count() from system.query_log where query like '%01231_log_queries_min_type/%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute;
set log_queries_min_type='EXCEPTION_WHILE_PROCESSING';
select '01231_log_queries_min_type/', max(number) from system.numbers limit 1e6 settings max_rows_to_read='100K'; -- { serverError 158; }
system flush logs;
select count() from system.query_log where query like '%01231_log_queries_min_type/%' and query not like '%system.query_log%';
select count() from system.query_log where query like '%01231_log_queries_min_type/%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute;

View File

@ -0,0 +1,21 @@
DROP TABLE IF EXISTS data_01283;
CREATE TABLE data_01283 engine=MergeTree()
ORDER BY key
PARTITION BY key
AS SELECT number key FROM numbers(10);
SET log_queries=1;
SELECT * FROM data_01283 LIMIT 1 FORMAT Null;
SET log_queries=0;
SYSTEM FLUSH LOGS;
-- 1 for PullingAsyncPipelineExecutor::pull
-- 1 for AsynchronousBlockInputStream
SELECT
throwIf(count() != 1, 'no query was logged'),
throwIf(length(thread_ids) != 2, 'too many threads used')
FROM system.query_log
WHERE type = 'QueryFinish' AND query LIKE '%data_01283 LIMIT 1%'
GROUP BY thread_ids
FORMAT Null;

View File

@ -1,3 +1 @@
# -*- coding: utf-8 -*-
# REMOVE ME

View File

@ -7,15 +7,15 @@
- All pull-requests must be squash-merged or explicitly merged without rebase.
- All pull-requests to master must have at least one label prefixed with `pr-`.
- Labels that require pull-request to be backported must be red colored (#ff0000).
- Stable branch name must be of form `YY.NUMBER`.
- All stable branches must be forked directly from the master branch and never be merged back,
- Release branch name must be of form `YY.NUMBER`.
- All release branches must be forked directly from the master branch and never be merged back,
or merged with any other branches based on the master branch (including master branch itself).
Output of this script:
- Commits without references from pull-requests.
- Pull-requests to master without proper labels.
- Pull-requests that need to be backported, with statuses per stable branch.
- Pull-requests that need to be backported, with statuses per release branch.
'''
@ -29,7 +29,7 @@ import sys
try:
from termcolor import colored # `pip install termcolor`
except ImportError:
sys.exit("Package 'termcolor' not found. Try run: `pip3 install termcolor`")
sys.exit("Package 'termcolor' not found. Try run: `pip3 install [--user] termcolor`")
CHECK_MARK = colored('🗸', 'green')
@ -45,8 +45,6 @@ parser.add_argument('--repo', '-r', type=str, default='', metavar='PATH',
help='path to the root of the ClickHouse repository')
parser.add_argument('--remote', type=str, default='origin',
help='remote name of the "ClickHouse/ClickHouse" upstream')
parser.add_argument('-n', type=int, default=3, dest='number',
help='number of last stable branches to consider')
parser.add_argument('--token', type=str, required=True,
help='token for Github access')
parser.add_argument('--login', type=str,
@ -54,31 +52,46 @@ parser.add_argument('--login', type=str,
parser.add_argument('--auto-label', action='store_true', dest='autolabel', default=True,
help='try to automatically parse PR description and put labels')
# Either select last N release branches, or specify them manually.
group = parser.add_mutually_exclusive_group(required=True)
group.add_argument('-n', type=int, default=3, dest='number',
help='number of last release branches to consider')
group.add_argument('--branch', type=str, action='append', metavar='BRANCH',
help='specific release branch name to consider')
args = parser.parse_args()
github = query.Query(args.token, 30)
repo = local.Local(args.repo, args.remote, github.get_default_branch())
stables = repo.get_stables()[-args.number:] # [(branch name, base)]
if not stables:
if not args.branch:
release_branches = repo.get_release_branches()[-args.number:] # [(branch name, base)]
else:
release_branches = []
all_release_branches = repo.get_release_branches()
for branch in all_release_branches:
if branch[0] in args.branch:
release_branches.append(branch)
if not release_branches:
sys.exit('No release branches found!')
else:
print('Found release branches:')
for stable in stables:
print(f'{CHECK_MARK} {stable[0]} forked from {stable[1]}')
for branch in release_branches:
print(f'{CHECK_MARK} {branch[0]} forked from {branch[1]}')
first_commit = stables[0][1]
first_commit = release_branches[0][1]
pull_requests = github.get_pull_requests(first_commit, args.login)
good_commits = set(pull_request['mergeCommit']['oid'] for pull_request in pull_requests)
bad_commits = [] # collect and print them in the end
from_commit = repo.get_head_commit()
for i in reversed(range(len(stables))):
for commit in repo.iterate(from_commit, stables[i][1]):
for i in reversed(range(len(release_branches))):
for commit in repo.iterate(from_commit, release_branches[i][1]):
if str(commit) not in good_commits and commit.author.name != 'robot-clickhouse':
bad_commits.append(commit)
from_commit = stables[i][1]
from_commit = release_branches[i][1]
members = set(github.get_members("ClickHouse", "ClickHouse"))
def print_responsible(pull_request):
@ -146,22 +159,22 @@ if need_backporting:
no_backport_labeled = set()
wait = set()
for stable in stables:
if repo.comparator(stable[1]) < repo.comparator(pull_request['mergeCommit']['oid']):
targets.append(stable[0])
for branch in release_branches:
if repo.comparator(branch[1]) < repo.comparator(pull_request['mergeCommit']['oid']):
targets.append(branch[0])
# FIXME: compatibility logic - check for a manually set label, that indicates status 'backported'.
# FIXME: O(n²) - no need to iterate all labels for every `stable`
# FIXME: O(n²) - no need to iterate all labels for every `branch`
for label in github.get_labels(pull_request):
if re_vlabel.match(label['name']) or re_vlabel_backported.match(label['name']):
if f'v{stable[0]}' == label['name'] or f'v{stable[0]}-backported' == label['name']:
backport_labeled.add(stable[0])
if f'v{branch[0]}' == label['name'] or f'v{branch[0]}-backported' == label['name']:
backport_labeled.add(branch[0])
if re_vlabel_conflicts.match(label['name']):
if f'v{stable[0]}-conflicts' == label['name']:
conflict_labeled.add(stable[0])
if f'v{branch[0]}-conflicts' == label['name']:
conflict_labeled.add(branch[0])
if re_vlabel_no_backport.match(label['name']):
if f'v{stable[0]}-no-backport' == label['name']:
no_backport_labeled.add(stable[0])
if f'v{branch[0]}-no-backport' == label['name']:
no_backport_labeled.add(branch[0])
for event in github.get_timeline(pull_request):
if(event['isCrossRepository'] or

View File

@ -1,7 +1,9 @@
# -*- coding: utf-8 -*-
# `pip install …`
import git # gitpython
try:
import git # `pip3 install gitpython`
except ImportError:
sys.exit("Package 'gitpython' not found. Try run: `pip3 install [--user] gitpython`")
import functools
import os
@ -11,7 +13,7 @@ import re
class Local:
'''Implements some useful methods atop of the local repository
'''
RE_STABLE_REF = re.compile(r'^refs/remotes/.+/\d+\.\d+$')
RE_RELEASE_BRANCH_REF = re.compile(r'^refs/remotes/.+/\d+\.\d+$')
def __init__(self, repo_path, remote_name, default_branch_name):
self._repo = git.Repo(repo_path, search_parent_directories=(not repo_path))
@ -42,16 +44,16 @@ class Local:
* head (git.Commit)).
List is sorted by commits in ascending order.
'''
def get_stables(self):
stables = []
def get_release_branches(self):
release_branches = []
for stable in [r for r in self._remote.refs if Local.RE_STABLE_REF.match(r.path)]:
base = self._repo.merge_base(self._default, self._repo.commit(stable))
for branch in [r for r in self._remote.refs if Local.RE_RELEASE_BRANCH_REF.match(r.path)]:
base = self._repo.merge_base(self._default, self._repo.commit(branch))
if not base:
print(f'Branch {stable.path} is not based on branch {self._default}. Ignoring.')
print(f'Branch {branch.path} is not based on branch {self._default}. Ignoring.')
elif len(base) > 1:
print(f'Branch {stable.path} has more than one base commit. Ignoring.')
print(f'Branch {branch.path} has more than one base commit. Ignoring.')
else:
stables.append((os.path.basename(stable.name), base[0]))
release_branches.append((os.path.basename(branch.name), base[0]))
return sorted(stables, key=lambda x : self.comparator(x[1]))
return sorted(release_branches, key=lambda x : self.comparator(x[1]))