Merge branch 'master' of github.com:ClickHouse/ClickHouse into Avogar-patch-10

This commit is contained in:
avogar 2024-08-22 10:43:23 +00:00
commit e73e8e7a08
29 changed files with 201 additions and 18 deletions

View File

@ -40,7 +40,13 @@ Every month we get together with the community (users, contributors, customers,
Keep an eye out for upcoming meetups and events around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `<at>` clickhouse `<dot>` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc.
The following upcoming meetups are featuring creator of ClickHouse & CTO, Alexey Milovidov:
* [ClickHouse Guangzhou User Group Meetup](https://mp.weixin.qq.com/s/GSvo-7xUoVzCsuUvlLTpCw) - August 25
* [San Francisco Meetup (Cloudflare)](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/302540575) - September 5
* [Raleigh Meetup (Deutsche Bank)](https://www.meetup.com/clickhouse-nc-meetup-group/events/302557230) - September 9
* [New York Meetup (Ramp)](https://www.meetup.com/clickhouse-new-york-user-group/events/302575342) - September 10
* [Chicago Meetup (Jump Capital)](https://lu.ma/43tvmrfw) - September 12
## Recent Recordings
* **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments"

View File

@ -155,6 +155,8 @@ SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'tag 1';
SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'tag 2';
```
To remove only entries with tag `tag` from the query cache, you can use statement `SYSTEM DROP QUERY CACHE TAG 'tag'`.
ClickHouse reads table data in blocks of [max_block_size](settings/settings.md#setting-max_block_size) rows. Due to filtering, aggregation,
etc., result blocks are typically much smaller than 'max_block_size' but there are also cases where they are much bigger. Setting
[query_cache_squash_partial_results](settings/settings.md#query-cache-squash-partial-results) (enabled by default) controls if result blocks

View File

@ -136,7 +136,13 @@ The compiled expression cache is enabled/disabled with the query/user/profile-le
## DROP QUERY CACHE
```sql
SYSTEM DROP QUERY CACHE;
SYSTEM DROP QUERY CACHE TAG '<tag>'
````
Clears the [query cache](../../operations/query-cache.md).
If a tag is specified, only query cache entries with the specified tag are deleted.
## DROP FORMAT SCHEMA CACHE {#system-drop-schema-format}

View File

@ -197,6 +197,12 @@ public:
cache_policy->remove(key);
}
void remove(std::function<bool(const Key&, const MappedPtr &)> predicate)
{
std::lock_guard lock(mutex);
cache_policy->remove(predicate);
}
size_t sizeInBytes() const
{
std::lock_guard lock(mutex);

View File

@ -55,6 +55,7 @@ public:
virtual void set(const Key & key, const MappedPtr & mapped) = 0;
virtual void remove(const Key & key) = 0;
virtual void remove(std::function<bool(const Key & key, const MappedPtr & mapped)> predicate) = 0;
virtual void clear() = 0;
virtual std::vector<KeyMapped> dump() const = 0;

View File

@ -79,6 +79,22 @@ public:
cells.erase(it);
}
void remove(std::function<bool(const Key &, const MappedPtr &)> predicate) override
{
for (auto it = cells.begin(); it != cells.end();)
{
if (predicate(it->first, it->second.value))
{
Cell & cell = it->second;
current_size_in_bytes -= cell.size;
queue.erase(cell.queue_iterator);
it = cells.erase(it);
}
else
++it;
}
}
MappedPtr get(const Key & key) override
{
auto it = cells.find(key);

View File

@ -95,6 +95,27 @@ public:
cells.erase(it);
}
void remove(std::function<bool(const Key &, const MappedPtr &)> predicate) override
{
for (auto it = cells.begin(); it != cells.end();)
{
if (predicate(it->first, it->second.value))
{
auto & cell = it->second;
current_size_in_bytes -= cell.size;
if (cell.is_protected)
current_protected_size -= cell.size;
auto & queue = cell.is_protected ? protected_queue : probationary_queue;
queue.erase(cell.queue_iterator);
it = cells.erase(it);
}
else
++it;
}
}
MappedPtr get(const Key & key) override
{
auto it = cells.find(key);

View File

@ -145,6 +145,23 @@ public:
size_in_bytes -= sz;
}
void remove(std::function<bool(const Key &, const MappedPtr &)> predicate) override
{
for (auto it = cache.begin(); it != cache.end();)
{
if (predicate(it->first, it->second))
{
size_t sz = weight_function(*it->second);
if (it->first.user_id.has_value())
Base::user_quotas->decreaseActual(*it->first.user_id, sz);
it = cache.erase(it);
size_in_bytes -= sz;
}
else
++it;
}
}
MappedPtr get(const Key & key) override
{
auto it = cache.find(key);

View File

@ -149,6 +149,8 @@ std::unique_ptr<IDataType::SubstreamData> IDataType::getSubcolumnData(
ISerialization::EnumerateStreamsSettings settings;
settings.position_independent_encoding = false;
/// Don't enumerate dynamic subcolumns, they are handled separately.
settings.enumerate_dynamic_streams = false;
data.serialization->enumerateStreams(settings, callback_with_data, data);
if (!res && data.type->hasDynamicSubcolumnsData())

View File

@ -241,6 +241,10 @@ public:
{
SubstreamPath path;
bool position_independent_encoding = true;
/// If set to false, don't enumerate dynamic subcolumns
/// (such as dynamic types in Dynamic column or dynamic paths in JSON column).
/// It may be needed when dynamic subcolumns are processed separately.
bool enumerate_dynamic_streams = true;
};
virtual void enumerateStreams(

View File

@ -64,7 +64,7 @@ void SerializationDynamic::enumerateStreams(
const auto * deserialize_state = data.deserialize_state ? checkAndGetState<DeserializeBinaryBulkStateDynamic>(data.deserialize_state) : nullptr;
/// If column is nullptr and we don't have deserialize state yet, nothing to enumerate as we don't have any variants.
if (!column_dynamic && !deserialize_state)
if (!settings.enumerate_dynamic_streams || (!column_dynamic && !deserialize_state))
return;
const auto & variant_type = column_dynamic ? column_dynamic->getVariantInfo().variant_type : checkAndGetState<DeserializeBinaryBulkStateDynamicStructure>(deserialize_state->structure_state)->variant_type;

View File

@ -130,7 +130,7 @@ void SerializationObject::enumerateStreams(EnumerateStreamsSettings & settings,
}
/// If column or deserialization state was provided, iterate over dynamic paths,
if (column_object || structure_state)
if (settings.enumerate_dynamic_streams && (column_object || structure_state))
{
/// Enumerate dynamic paths in sorted order for consistency.
const auto * dynamic_paths = column_object ? &column_object->getDynamicPaths() : nullptr;

View File

@ -6,7 +6,6 @@
#include <Columns/ColumnString.h>
#include <Functions/LowerUpperImpl.h>
#include <base/find_symbols.h>
#include <unicode/unistr.h>
#include <Common/StringUtils.h>
@ -43,7 +42,7 @@ struct LowerUpperUTF8Impl
String output;
size_t curr_offset = 0;
for (size_t i = 0; i < offsets.size(); ++i)
for (size_t i = 0; i < input_rows_count; ++i)
{
const auto * data_start = reinterpret_cast<const char *>(&data[offsets[i - 1]]);
size_t size = offsets[i] - offsets[i - 1];
@ -57,13 +56,15 @@ struct LowerUpperUTF8Impl
output.clear();
input.toUTF8String(output);
/// For valid UTF-8 input strings, ICU sometimes produces output with extra '\0's at the end. Only the data before the first
/// '\0' is valid. It the input is not valid UTF-8, then the behavior of lower/upperUTF8 is undefined by definition. In this
/// For valid UTF-8 input strings, ICU sometimes produces output with an extra '\0 at the end. Only the data before that
/// '\0' is valid. If the input is not valid UTF-8, then the behavior of lower/upperUTF8 is undefined by definition. In this
/// case, the behavior is also reasonable.
const char * res_end = find_last_not_symbols_or_null<'\0'>(output.data(), output.data() + output.size());
size_t valid_size = res_end ? res_end - output.data() + 1 : 0;
size_t valid_size = output.size();
if (!output.empty() && output.back() == '\0')
--valid_size;
res_data.resize(curr_offset + valid_size + 1);
memcpy(&res_data[curr_offset], output.data(), valid_size);
res_data[curr_offset + valid_size] = 0;

View File

@ -619,9 +619,18 @@ QueryCache::Writer QueryCache::createWriter(const Key & key, std::chrono::millis
return Writer(cache, key, max_entry_size_in_bytes, max_entry_size_in_rows, min_query_runtime, squash_partial_results, max_block_size);
}
void QueryCache::clear()
void QueryCache::clear(const std::optional<String> & tag)
{
cache.clear();
if (tag)
{
auto predicate = [tag](const Key & key, const Cache::MappedPtr &) { return key.tag == tag.value(); };
cache.remove(predicate);
}
else
{
cache.clear();
}
std::lock_guard lock(mutex);
times_executed.clear();
}

View File

@ -211,7 +211,7 @@ public:
Reader createReader(const Key & key);
Writer createWriter(const Key & key, std::chrono::milliseconds min_query_runtime, bool squash_partial_results, size_t max_block_size, size_t max_query_cache_size_in_bytes_quota, size_t max_query_cache_entries_quota);
void clear();
void clear(const std::optional<String> & tag);
size_t sizeInBytes() const;
size_t count() const;

View File

@ -3228,12 +3228,12 @@ QueryCachePtr Context::getQueryCache() const
return shared->query_cache;
}
void Context::clearQueryCache() const
void Context::clearQueryCache(const std::optional<String> & tag) const
{
std::lock_guard lock(shared->mutex);
if (shared->query_cache)
shared->query_cache->clear();
shared->query_cache->clear(tag);
}
void Context::clearCaches() const

View File

@ -1068,7 +1068,7 @@ public:
void setQueryCache(size_t max_size_in_bytes, size_t max_entries, size_t max_entry_size_in_bytes, size_t max_entry_size_in_rows);
void updateQueryCacheConfiguration(const Poco::Util::AbstractConfiguration & config);
std::shared_ptr<QueryCache> getQueryCache() const;
void clearQueryCache() const;
void clearQueryCache(const std::optional<String> & tag) const;
/** Clear the caches of the uncompressed blocks and marks.
* This is usually done when renaming tables, changing the type of columns, deleting a table.

View File

@ -369,9 +369,12 @@ BlockIO InterpreterSystemQuery::execute()
system_context->clearMMappedFileCache();
break;
case Type::DROP_QUERY_CACHE:
{
getContext()->checkAccess(AccessType::SYSTEM_DROP_QUERY_CACHE);
getContext()->clearQueryCache();
getContext()->clearQueryCache(query.query_cache_tag);
break;
}
case Type::DROP_COMPILED_EXPRESSION_CACHE:
#if USE_EMBEDDED_COMPILER
getContext()->checkAccess(AccessType::SYSTEM_DROP_COMPILED_EXPRESSION_CACHE);

View File

@ -131,6 +131,8 @@ public:
String disk;
UInt64 seconds{};
std::optional<String> query_cache_tag;
String filesystem_cache_name;
std::string key_to_drop;
std::optional<size_t> offset_to_drop;

View File

@ -470,6 +470,7 @@ namespace DB
MR_MACROS(TABLE_OVERRIDE, "TABLE OVERRIDE") \
MR_MACROS(TABLE, "TABLE") \
MR_MACROS(TABLES, "TABLES") \
MR_MACROS(TAG, "TAG") \
MR_MACROS(TAGS, "TAGS") \
MR_MACROS(TAGS_INNER_UUID, "TAGS INNER UUID") \
MR_MACROS(TEMPORARY_TABLE, "TEMPORARY TABLE") \

View File

@ -471,6 +471,16 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
res->seconds = seconds->as<ASTLiteral>()->value.safeGet<UInt64>();
break;
}
case Type::DROP_QUERY_CACHE:
{
ParserLiteral tag_parser;
ASTPtr ast;
if (ParserKeyword{Keyword::TAG}.ignore(pos, expected) && tag_parser.parse(pos, ast, expected))
res->query_cache_tag = std::make_optional<String>(ast->as<ASTLiteral>()->value.safeGet<String>());
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
break;
}
case Type::DROP_FILESYSTEM_CACHE:
{
ParserLiteral path_parser;

View File

@ -80,6 +80,9 @@ fi
export IS_FLAKY_CHECK=0
# Export NUM_TRIES so python scripts will see its value as env variable
export NUM_TRIES
# For flaky check we also enable thread fuzzer
if [ "$NUM_TRIES" -gt "1" ]; then
export IS_FLAKY_CHECK=1

View File

@ -26,3 +26,4 @@
1
1
1
2

View File

@ -38,3 +38,6 @@ select lowerUTF8('ır') = 'ır';
-- German language
select upper('öäüß') = 'öäüß';
select lower('ÖÄÜẞ') = 'ÖÄÜẞ';
-- Bug 68680
SELECT lengthUTF8(lowerUTF8('Ä\0'));

View File

@ -6,8 +6,17 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
# Sanity check to ensure that the server is up and running
for _ in {1..10}; do
echo 'SELECT 1' | ${CLICKHOUSE_CURL_COMMAND} -s "${CLICKHOUSE_URL}" --data-binary @- > /dev/null
if [ $? -eq 0 ]; then
break
fi
sleep 1
done
CURL_OUTPUT=$(echo 'SELECT 1 + sleepEachRow(0.00002) FROM numbers(100000)' | \
${CLICKHOUSE_CURL_COMMAND} -vsS "${CLICKHOUSE_URL}&wait_end_of_query=1&send_progress_in_http_headers=0&max_execution_time=1" --data-binary @- 2>&1)
${CLICKHOUSE_CURL_COMMAND} --max-time 3 -vsS "${CLICKHOUSE_URL}&wait_end_of_query=1&send_progress_in_http_headers=0&max_execution_time=1" --data-binary @- 2>&1)
READ_ROWS=$(echo "${CURL_OUTPUT}" | \
grep 'X-ClickHouse-Summary' | \
@ -20,6 +29,7 @@ then
echo "Read rows in summary is not zero"
else
echo "Read rows in summary is zero!"
echo "${CURL_OUTPUT}"
fi
# Check that the response code is correct too

View File

@ -1,3 +1,17 @@
Cache query result in query cache
1
1
DROP entries with a certain tag, no entry will match
1
After a full DROP, the cache is empty now
0
Cache query result with different or no tag in query cache
1
1
1
2
4
DROP entries with certain tags
2
1
0

View File

@ -4,10 +4,31 @@
-- (it's silly to use what will be tested below but we have to assume other tests cluttered the query cache)
SYSTEM DROP QUERY CACHE;
-- Cache query result in query cache
SELECT 'Cache query result in query cache';
SELECT 1 SETTINGS use_query_cache = true;
SELECT count(*) FROM system.query_cache;
-- No query results are cached after DROP
SELECT 'DROP entries with a certain tag, no entry will match';
SYSTEM DROP QUERY CACHE TAG 'tag';
SELECT count(*) FROM system.query_cache;
SELECT 'After a full DROP, the cache is empty now';
SYSTEM DROP QUERY CACHE;
SELECT count(*) FROM system.query_cache;
-- More tests for DROP with tags:
SELECT 'Cache query result with different or no tag in query cache';
SELECT 1 SETTINGS use_query_cache = true;
SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'abc';
SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'def';
SELECT 2 SETTINGS use_query_cache = true;
SELECT count(*) FROM system.query_cache;
SELECT 'DROP entries with certain tags';
SYSTEM DROP QUERY CACHE TAG '';
SELECT count(*) FROM system.query_cache;
SYSTEM DROP QUERY CACHE TAG 'def';
SELECT count(*) FROM system.query_cache;
SYSTEM DROP QUERY CACHE TAG 'abc';
SELECT count(*) FROM system.query_cache;

View File

@ -0,0 +1,15 @@
\N
\N
\N
\N
\N
str_0
str_1
str_2
str_3
str_4
\N
\N
\N
\N
\N

View File

@ -0,0 +1,9 @@
set allow_experimental_json_type=1;
drop table if exists test;
create table test (json JSON) engine=Memory;
insert into test select toJSONString(map('a', 'str_' || number)) from numbers(5);
select json.a.String from test;
select json.a.:String from test;
select json.a.UInt64 from test;
drop table test;