Merge branch 'fix_to_string_nullable_timezone' of github.com:yariks5s/ClickHouse into fix_to_string_nullable_timezone

This commit is contained in:
yariks5s 2024-01-25 12:02:49 +00:00
commit 1620c069b7
5 changed files with 69 additions and 4 deletions

View File

@ -56,6 +56,32 @@ takeFromRange(const MarkRange & range, size_t min_number_of_marks, size_t & curr
current_marks_amount += range_we_take.getNumberOfMarks();
return range_we_take.getNumberOfMarks();
}
void sortResponseRanges(RangesInDataPartsDescription & result)
{
std::ranges::sort(result, [](const auto & lhs, const auto & rhs) { return lhs.info < rhs.info; });
RangesInDataPartsDescription new_result;
/// Aggregate ranges for each part within a single entry
for (auto & ranges_in_part : result)
{
if (new_result.empty() || new_result.back().info != ranges_in_part.info)
new_result.push_back(RangesInDataPartDescription{.info = ranges_in_part.info});
new_result.back().ranges.insert(
new_result.back().ranges.end(),
std::make_move_iterator(ranges_in_part.ranges.begin()),
std::make_move_iterator(ranges_in_part.ranges.end()));
ranges_in_part.ranges.clear();
}
/// Sort ranges for each part
for (auto & ranges_in_part : new_result)
std::sort(ranges_in_part.ranges.begin(), ranges_in_part.ranges.end());
result = std::move(new_result);
}
}
namespace ProfileEvents
@ -775,6 +801,8 @@ ParallelReadResponse DefaultCoordinator::handleRequest(ParallelReadRequest reque
}
}
sortResponseRanges(response.description);
LOG_DEBUG(
log,
"Going to respond to replica {} with {}; mine_marks={}, stolen_by_hash={}, stolen_rest={}",

View File

@ -1,12 +1,12 @@
#!/usr/bin/env python3
from collections import namedtuple
import fnmatch
import json
import time
from collections import namedtuple
from urllib.parse import quote
import requests # type: ignore
from lambda_shared.pr import TRUSTED_CONTRIBUTORS
from lambda_shared.token import get_cached_access_token
@ -129,7 +129,7 @@ def _exec_post_with_retry(url, token, data=None):
def _get_pull_requests_from(repo_url, owner, branch, token):
url = f"{repo_url}/pulls?head={owner}:{branch}"
url = f"{repo_url}/pulls?head={quote(owner)}:{quote(branch)}"
return _exec_get_with_retry(url, token)

View File

@ -4,7 +4,7 @@ SELECT 1 SETTINGS max_execution_time=-Infinity; -- { clientError 72 };
-- Ok values
SELECT 1 SETTINGS max_execution_time=-0.5;
SELECT 1 SETTINGS max_execution_time=0.5;
SELECT 1 SETTINGS max_execution_time=5.5;
SELECT 1 SETTINGS max_execution_time=-1;
SELECT 1 SETTINGS max_execution_time=0.0;
SELECT 1 SETTINGS max_execution_time=-0.0;

View File

@ -0,0 +1,36 @@
#!/usr/bin/env bash
# Tags: no-parallel, no-random-settings, long
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
# Test assumes that the whole table is residing in the cache, but `hits_s3` has only 128Mi of cache.
# So we need to create a smaller table.
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS hits_s3_sampled"
$CLICKHOUSE_CLIENT -q "CREATE TABLE hits_s3_sampled AS test.hits_s3"
$CLICKHOUSE_CLIENT -q "INSERT INTO hits_s3_sampled SELECT * FROM test.hits_s3 SAMPLE 0.01"
$CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE hits_s3_sampled FINAL"
$CLICKHOUSE_CLIENT -q "SYSTEM DROP FILESYSTEM CACHE"
# Warm up the cache
$CLICKHOUSE_CLIENT -q "SELECT * FROM hits_s3_sampled WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null"
$CLICKHOUSE_CLIENT -q "SELECT * FROM hits_s3_sampled WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null"
query_id=02906_read_from_cache_$RANDOM
$CLICKHOUSE_CLIENT --query_id ${query_id} -q "SELECT * FROM hits_s3_sampled WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null"
$CLICKHOUSE_CLIENT -nq "
SYSTEM FLUSH LOGS;
-- AsynchronousReaderIgnoredBytes = 0: no seek-avoiding happened
-- CachedReadBufferReadFromSourceBytes = 0: sanity check to ensure we read only from cache
SELECT ProfileEvents['AsynchronousReaderIgnoredBytes'], ProfileEvents['CachedReadBufferReadFromSourceBytes']
FROM system.query_log
WHERE query_id = '$query_id' AND type = 'QueryFinish' AND event_date >= yesterday() AND current_database = currentDatabase()
"
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS hits_s3_sampled"