Merge remote-tracking branch 'blessed/master' into argmin_optimization

This commit is contained in:
Raúl Marín 2024-01-19 14:43:44 +01:00
commit 3739d46817
124 changed files with 1736 additions and 1127 deletions

View File

@ -8,7 +8,6 @@ on: # yamllint disable-line rule:truthy
schedule:
- cron: '0 */6 * * *'
workflow_dispatch:
workflow_call:
jobs:
KeeperJepsenRelease:
uses: ./.github/workflows/reusable_simple_job.yml

View File

@ -966,13 +966,20 @@ jobs:
#############################################################################################
###################################### JEPSEN TESTS #########################################
#############################################################################################
# This is special test NOT INCLUDED in FinishCheck
# When it's skipped, all dependent tasks will be skipped too.
# DO NOT add it there
Jepsen:
# This is special test NOT INCLUDED in FinishCheck
# When it's skipped, all dependent tasks will be skipped too.
# DO NOT add it there
if: ${{ !failure() && !cancelled() && contains(github.event.pull_request.labels.*.name, 'jepsen-test') }}
# we need concurrency as the job uses dedicated instances in the cloud
concurrency:
group: jepsen
if: ${{ !failure() && !cancelled() }}
needs: [RunConfig, BuilderBinRelease]
uses: ./.github/workflows/jepsen.yml
uses: ./.github/workflows/reusable_test.yml
with:
test_name: ClickHouse Keeper Jepsen
runner_type: style-checker
data: ${{ needs.RunConfig.outputs.data }}
#############################################################################################
####################################### libFuzzer ###########################################
#############################################################################################

View File

@ -58,6 +58,7 @@ jobs:
- name: Apply sparse checkout for contrib # in order to check that it doesn't break build
# This step is done in GITHUB_WORKSPACE,
# because it's broken in REPO_COPY for some reason
# See also update-submodules.sh
if: ${{ env.BUILD_SPARSE_CHECKOUT == 'true' }}
run: |
rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed'

View File

@ -58,6 +58,8 @@ jobs:
env:
GITHUB_JOB_OVERRIDDEN: ${{inputs.test_name}}
steps:
- name: DebugInfo
uses: hmarr/debug-action@a701ed95a46e6f2fb0df25e1a558c16356fae35a
- name: Check out repository code
uses: ClickHouse/checkout@v1
with:

View File

@ -6,9 +6,15 @@ SCRIPT_DIR=$(dirname "${SCRIPT_PATH}")
GIT_DIR=$(git -C "$SCRIPT_DIR" rev-parse --show-toplevel)
cd $GIT_DIR
# Exclude from contribs some garbage subdirs that we don't need.
# It reduces the checked out files size about 3 times and therefore speeds up indexing in IDEs and searching.
# NOTE .git/ still contains everything that we don't check out (although, it's compressed)
# See also https://git-scm.com/docs/git-sparse-checkout
contrib/sparse-checkout/setup-sparse-checkout.sh
git submodule init
git submodule sync
# NOTE: do not use --remote for `git submodule update`[1] command, since the submodule references to the specific commit SHA1 in the subproject.
# It may cause unexpected behavior. Instead you need to commit a new SHA1 for a submodule.
#

View File

@ -235,6 +235,17 @@ clickhouse-client -q "system flush logs" ||:
# stop logs replication to make it possible to dump logs tables via clickhouse-local
stop_logs_replication
# Try to get logs while server is running
successfuly_saved=0
for table in query_log zookeeper_log trace_log transactions_info_log
do
clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst || successfuly_saved=$((successfuly_saved+$?))
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst || successfuly_saved=$((successfuly_saved+$?))
clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst || successfuly_saved=$((successfuly_saved+$?))
fi
done
# Stop server so we can safely read data with clickhouse-local.
# Why do we read data with clickhouse-local?
# Because it's the simplest way to read it when server has crashed.
@ -254,21 +265,25 @@ if [[ -n "$USE_S3_STORAGE_FOR_MERGE_TREE" ]] && [[ "$USE_S3_STORAGE_FOR_MERGE_TR
data_path_config="--config-file=/etc/clickhouse-server/config.xml"
fi
# Compress tables.
#
# NOTE:
# - that due to tests with s3 storage we cannot use /var/lib/clickhouse/data
# directly
# - even though ci auto-compress some files (but not *.tsv) it does this only
# for files >64MB, we want this files to be compressed explicitly
for table in query_log zookeeper_log trace_log transactions_info_log
do
clickhouse-local "$data_path_config" --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst ||:
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
clickhouse-local --path /var/lib/clickhouse1/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst ||:
clickhouse-local --path /var/lib/clickhouse2/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst ||:
fi
done
# If server crashed dump system logs with clickhouse-local
if [ $successfuly_saved -ne 0 ]; then
# Compress tables.
#
# NOTE:
# - that due to tests with s3 storage we cannot use /var/lib/clickhouse/data
# directly
# - even though ci auto-compress some files (but not *.tsv) it does this only
# for files >64MB, we want this files to be compressed explicitly
for table in query_log zookeeper_log trace_log transactions_info_log
do
clickhouse-local "$data_path_config" --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst ||:
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
clickhouse-local --path /var/lib/clickhouse1/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst ||:
clickhouse-local --path /var/lib/clickhouse2/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst ||:
fi
done
fi
# Also export trace log in flamegraph-friendly format.
for trace_type in CPU Memory Real

View File

@ -63,7 +63,7 @@ if __name__ == "__main__":
Let's invoke the script and have it generate 10 random strings:
```sql
SELECT * FROM executable('my_script.py', TabSeparated, 'id UInt32, random String', (SELECT 10))
SELECT * FROM executable('generate_random.py', TabSeparated, 'id UInt32, random String', (SELECT 10))
```
The response looks like:

View File

@ -290,6 +290,11 @@ void LocalServer::cleanup()
{
connection.reset();
/// Suggestions are loaded async in a separate thread and it can use global context.
/// We should reset it before resetting global_context.
if (suggest)
suggest.reset();
if (global_context)
{
global_context->shutdown();

View File

@ -1,4 +1,4 @@
#include "AutoFinalOnQueryPass.h"
#include <Analyzer/Passes/AutoFinalOnQueryPass.h>
#include <Storages/IStorage.h>

View File

@ -8,14 +8,12 @@
#include <Analyzer/ConstantNode.h>
#include <Analyzer/Passes/CNF.h>
#include <Analyzer/Utils.h>
#include <Analyzer/HashUtils.h>
#include <Storages/IStorage.h>
#include <Functions/FunctionFactory.h>
#include "Analyzer/HashUtils.h"
#include "Analyzer/IQueryTreeNode.h"
#include "Interpreters/ComparisonGraph.h"
#include "base/types.h"
#include <Interpreters/ComparisonGraph.h>
namespace DB
{

View File

@ -75,6 +75,7 @@ public:
{
/// Replace `length(array_argument)` with `array_argument.size0`
column.name += ".size0";
column.type = std::make_shared<DataTypeUInt64>();
node = std::make_shared<ColumnNode>(column, column_source);
}
@ -109,6 +110,7 @@ public:
{
/// Replace `isNull(nullable_argument)` with `nullable_argument.null`
column.name += ".null";
column.type = std::make_shared<DataTypeUInt8>();
node = std::make_shared<ColumnNode>(column, column_source);
}

View File

@ -2980,6 +2980,8 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromStorage(
if (!result_expression)
{
if (can_be_not_found)
return {};
std::unordered_set<Identifier> valid_identifiers;
collectTableExpressionValidIdentifiersForTypoCorrection(identifier,
table_expression_node,

View File

@ -1,4 +1,4 @@
#include "UniqToCountPass.h"
#include <Analyzer/Passes/UniqToCountPass.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/IAggregateFunction.h>

View File

@ -118,18 +118,18 @@ ConnectionPoolWithFailover::Status ConnectionPoolWithFailover::getStatus() const
return result;
}
std::vector<IConnectionPool::Entry> ConnectionPoolWithFailover::getMany(
const ConnectionTimeouts & timeouts,
const Settings & settings,
PoolMode pool_mode,
AsyncCallback async_callback,
std::optional<bool> skip_unavailable_endpoints,
GetPriorityForLoadBalancing::Func priority_func)
std::vector<IConnectionPool::Entry> ConnectionPoolWithFailover::getMany(const ConnectionTimeouts & timeouts,
const Settings & settings,
PoolMode pool_mode,
AsyncCallback async_callback,
std::optional<bool> skip_unavailable_endpoints)
{
TryGetEntryFunc try_get_entry = [&](NestedPool & pool, std::string & fail_message)
{ return tryGetEntry(pool, timeouts, fail_message, settings, nullptr, async_callback); };
{
return tryGetEntry(pool, timeouts, fail_message, settings, nullptr, async_callback);
};
std::vector<TryResult> results = getManyImpl(settings, pool_mode, try_get_entry, skip_unavailable_endpoints, priority_func);
std::vector<TryResult> results = getManyImpl(settings, pool_mode, try_get_entry, skip_unavailable_endpoints);
std::vector<Entry> entries;
entries.reserve(results.size());
@ -153,17 +153,17 @@ std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::g
std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::getManyChecked(
const ConnectionTimeouts & timeouts,
const Settings & settings,
PoolMode pool_mode,
const Settings & settings, PoolMode pool_mode,
const QualifiedTableName & table_to_check,
AsyncCallback async_callback,
std::optional<bool> skip_unavailable_endpoints,
GetPriorityForLoadBalancing::Func priority_func)
std::optional<bool> skip_unavailable_endpoints)
{
TryGetEntryFunc try_get_entry = [&](NestedPool & pool, std::string & fail_message)
{ return tryGetEntry(pool, timeouts, fail_message, settings, &table_to_check, async_callback); };
{
return tryGetEntry(pool, timeouts, fail_message, settings, &table_to_check, async_callback);
};
return getManyImpl(settings, pool_mode, try_get_entry, skip_unavailable_endpoints, priority_func);
return getManyImpl(settings, pool_mode, try_get_entry, skip_unavailable_endpoints);
}
ConnectionPoolWithFailover::Base::GetPriorityFunc ConnectionPoolWithFailover::makeGetPriorityFunc(const Settings & settings)
@ -175,16 +175,14 @@ ConnectionPoolWithFailover::Base::GetPriorityFunc ConnectionPoolWithFailover::ma
}
std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::getManyImpl(
const Settings & settings,
PoolMode pool_mode,
const TryGetEntryFunc & try_get_entry,
std::optional<bool> skip_unavailable_endpoints,
GetPriorityForLoadBalancing::Func priority_func)
const Settings & settings,
PoolMode pool_mode,
const TryGetEntryFunc & try_get_entry,
std::optional<bool> skip_unavailable_endpoints)
{
if (nested_pools.empty())
throw DB::Exception(
DB::ErrorCodes::ALL_CONNECTION_TRIES_FAILED,
"Cannot get connection from ConnectionPoolWithFailover cause nested pools are empty");
throw DB::Exception(DB::ErrorCodes::ALL_CONNECTION_TRIES_FAILED,
"Cannot get connection from ConnectionPoolWithFailover cause nested pools are empty");
if (!skip_unavailable_endpoints.has_value())
skip_unavailable_endpoints = settings.skip_unavailable_shards;
@ -205,13 +203,14 @@ std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::g
else
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unknown pool allocation mode");
if (!priority_func)
priority_func = makeGetPriorityFunc(settings);
GetPriorityFunc get_priority = makeGetPriorityFunc(settings);
UInt64 max_ignored_errors = settings.distributed_replica_max_ignored_errors.value;
bool fallback_to_stale_replicas = settings.fallback_to_stale_replicas_for_distributed_queries.value;
return Base::getMany(min_entries, max_entries, max_tries, max_ignored_errors, fallback_to_stale_replicas, try_get_entry, priority_func);
return Base::getMany(min_entries, max_entries, max_tries,
max_ignored_errors, fallback_to_stale_replicas,
try_get_entry, get_priority);
}
ConnectionPoolWithFailover::TryResult
@ -252,14 +251,11 @@ ConnectionPoolWithFailover::tryGetEntry(
return result;
}
std::vector<ConnectionPoolWithFailover::Base::ShuffledPool>
ConnectionPoolWithFailover::getShuffledPools(const Settings & settings, GetPriorityForLoadBalancing::Func priority_func)
std::vector<ConnectionPoolWithFailover::Base::ShuffledPool> ConnectionPoolWithFailover::getShuffledPools(const Settings & settings)
{
if (!priority_func)
priority_func = makeGetPriorityFunc(settings);
GetPriorityFunc get_priority = makeGetPriorityFunc(settings);
UInt64 max_ignored_errors = settings.distributed_replica_max_ignored_errors.value;
return Base::getShuffledPools(max_ignored_errors, priority_func);
return Base::getShuffledPools(max_ignored_errors, get_priority);
}
}

View File

@ -54,13 +54,10 @@ public:
/** Allocates up to the specified number of connections to work.
* Connections provide access to different replicas of one shard.
*/
std::vector<Entry> getMany(
const ConnectionTimeouts & timeouts,
const Settings & settings,
PoolMode pool_mode,
AsyncCallback async_callback = {},
std::optional<bool> skip_unavailable_endpoints = std::nullopt,
GetPriorityForLoadBalancing::Func priority_func = {});
std::vector<Entry> getMany(const ConnectionTimeouts & timeouts,
const Settings & settings, PoolMode pool_mode,
AsyncCallback async_callback = {},
std::optional<bool> skip_unavailable_endpoints = std::nullopt);
/// The same as getMany(), but return std::vector<TryResult>.
std::vector<TryResult> getManyForTableFunction(const ConnectionTimeouts & timeouts,
@ -72,13 +69,12 @@ public:
/// The same as getMany(), but check that replication delay for table_to_check is acceptable.
/// Delay threshold is taken from settings.
std::vector<TryResult> getManyChecked(
const ConnectionTimeouts & timeouts,
const Settings & settings,
PoolMode pool_mode,
const QualifiedTableName & table_to_check,
AsyncCallback async_callback = {},
std::optional<bool> skip_unavailable_endpoints = std::nullopt,
GetPriorityForLoadBalancing::Func priority_func = {});
const ConnectionTimeouts & timeouts,
const Settings & settings,
PoolMode pool_mode,
const QualifiedTableName & table_to_check,
AsyncCallback async_callback = {},
std::optional<bool> skip_unavailable_endpoints = std::nullopt);
struct NestedPoolStatus
{
@ -91,7 +87,7 @@ public:
using Status = std::vector<NestedPoolStatus>;
Status getStatus() const;
std::vector<Base::ShuffledPool> getShuffledPools(const Settings & settings, GetPriorityFunc priority_func = {});
std::vector<Base::ShuffledPool> getShuffledPools(const Settings & settings);
size_t getMaxErrorCup() const { return Base::max_error_cap; }
@ -100,16 +96,13 @@ public:
Base::updateSharedErrorCounts(shuffled_pools);
}
size_t getPoolSize() const { return Base::getPoolSize(); }
private:
/// Get the values of relevant settings and call Base::getMany()
std::vector<TryResult> getManyImpl(
const Settings & settings,
PoolMode pool_mode,
const TryGetEntryFunc & try_get_entry,
std::optional<bool> skip_unavailable_endpoints = std::nullopt,
GetPriorityForLoadBalancing::Func priority_func = {});
const Settings & settings,
PoolMode pool_mode,
const TryGetEntryFunc & try_get_entry,
std::optional<bool> skip_unavailable_endpoints = std::nullopt);
/// Try to get a connection from the pool and check that it is good.
/// If table_to_check is not null and the check is enabled in settings, check that replication delay
@ -122,7 +115,7 @@ private:
const QualifiedTableName * table_to_check = nullptr,
AsyncCallback async_callback = {});
GetPriorityForLoadBalancing::Func makeGetPriorityFunc(const Settings & settings);
GetPriorityFunc makeGetPriorityFunc(const Settings & settings);
GetPriorityForLoadBalancing get_priority_load_balancing;
};

View File

@ -28,18 +28,16 @@ HedgedConnections::HedgedConnections(
const ThrottlerPtr & throttler_,
PoolMode pool_mode,
std::shared_ptr<QualifiedTableName> table_to_check_,
AsyncCallback async_callback,
GetPriorityForLoadBalancing::Func priority_func)
AsyncCallback async_callback)
: hedged_connections_factory(
pool_,
context_->getSettingsRef(),
timeouts_,
context_->getSettingsRef().connections_with_failover_max_tries.value,
context_->getSettingsRef().fallback_to_stale_replicas_for_distributed_queries.value,
context_->getSettingsRef().max_parallel_replicas.value,
context_->getSettingsRef().skip_unavailable_shards.value,
table_to_check_,
priority_func)
pool_,
context_->getSettingsRef(),
timeouts_,
context_->getSettingsRef().connections_with_failover_max_tries.value,
context_->getSettingsRef().fallback_to_stale_replicas_for_distributed_queries.value,
context_->getSettingsRef().max_parallel_replicas.value,
context_->getSettingsRef().skip_unavailable_shards.value,
table_to_check_)
, context(std::move(context_))
, settings(context->getSettingsRef())
, throttler(throttler_)

View File

@ -70,15 +70,13 @@ public:
size_t index;
};
HedgedConnections(
const ConnectionPoolWithFailoverPtr & pool_,
ContextPtr context_,
const ConnectionTimeouts & timeouts_,
const ThrottlerPtr & throttler,
PoolMode pool_mode,
std::shared_ptr<QualifiedTableName> table_to_check_ = nullptr,
AsyncCallback async_callback = {},
GetPriorityForLoadBalancing::Func priority_func = {});
HedgedConnections(const ConnectionPoolWithFailoverPtr & pool_,
ContextPtr context_,
const ConnectionTimeouts & timeouts_,
const ThrottlerPtr & throttler,
PoolMode pool_mode,
std::shared_ptr<QualifiedTableName> table_to_check_ = nullptr,
AsyncCallback async_callback = {});
void sendScalarsData(Scalars & data) override;

View File

@ -29,8 +29,7 @@ HedgedConnectionsFactory::HedgedConnectionsFactory(
bool fallback_to_stale_replicas_,
UInt64 max_parallel_replicas_,
bool skip_unavailable_shards_,
std::shared_ptr<QualifiedTableName> table_to_check_,
GetPriorityForLoadBalancing::Func priority_func)
std::shared_ptr<QualifiedTableName> table_to_check_)
: pool(pool_)
, timeouts(timeouts_)
, table_to_check(table_to_check_)
@ -40,7 +39,7 @@ HedgedConnectionsFactory::HedgedConnectionsFactory(
, max_parallel_replicas(max_parallel_replicas_)
, skip_unavailable_shards(skip_unavailable_shards_)
{
shuffled_pools = pool->getShuffledPools(settings_, priority_func);
shuffled_pools = pool->getShuffledPools(settings_);
for (auto shuffled_pool : shuffled_pools)
replicas.emplace_back(std::make_unique<ConnectionEstablisherAsync>(shuffled_pool.pool, &timeouts, settings_, log, table_to_check.get()));
}
@ -324,7 +323,8 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::processFinishedConnect
else
{
ShuffledPool & shuffled_pool = shuffled_pools[index];
LOG_INFO(log, "Connection failed at try №{}, reason: {}", (shuffled_pool.error_count + 1), fail_message);
LOG_WARNING(
log, "Connection failed at try №{}, reason: {}", (shuffled_pool.error_count + 1), fail_message);
ProfileEvents::increment(ProfileEvents::DistributedConnectionFailTry);
shuffled_pool.error_count = std::min(pool->getMaxErrorCup(), shuffled_pool.error_count + 1);

View File

@ -53,8 +53,7 @@ public:
bool fallback_to_stale_replicas_,
UInt64 max_parallel_replicas_,
bool skip_unavailable_shards_,
std::shared_ptr<QualifiedTableName> table_to_check_ = nullptr,
GetPriorityForLoadBalancing::Func priority_func = {});
std::shared_ptr<QualifiedTableName> table_to_check_ = nullptr);
/// Create and return active connections according to pool_mode.
std::vector<Connection *> getManyConnections(PoolMode pool_mode, AsyncCallback async_callback = {});

View File

@ -590,6 +590,7 @@
M(708, ILLEGAL_STATISTIC) \
M(709, CANNOT_GET_REPLICATED_DATABASE_SNAPSHOT) \
M(710, FAULT_INJECTED) \
M(711, FILECACHE_ACCESS_DENIED) \
\
M(999, KEEPER_EXCEPTION) \
M(1000, POCO_EXCEPTION) \

View File

@ -9,8 +9,7 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
GetPriorityForLoadBalancing::Func
GetPriorityForLoadBalancing::getPriorityFunc(LoadBalancing load_balance, size_t offset, size_t pool_size) const
std::function<Priority(size_t index)> GetPriorityForLoadBalancing::getPriorityFunc(LoadBalancing load_balance, size_t offset, size_t pool_size) const
{
std::function<Priority(size_t index)> get_priority;
switch (load_balance)
@ -34,26 +33,19 @@ GetPriorityForLoadBalancing::getPriorityFunc(LoadBalancing load_balance, size_t
get_priority = [offset](size_t i) { return i != offset ? Priority{1} : Priority{0}; };
break;
case LoadBalancing::ROUND_ROBIN:
auto local_last_used = last_used % pool_size;
if (last_used >= pool_size)
last_used = 0;
++last_used;
// Example: pool_size = 5
// | local_last_used | i=0 | i=1 | i=2 | i=3 | i=4 |
// | 0 | 4 | 0 | 1 | 2 | 3 |
// | 1 | 3 | 4 | 0 | 1 | 2 |
// | 2 | 2 | 3 | 4 | 0 | 1 |
// | 3 | 1 | 2 | 3 | 4 | 0 |
// | 4 | 0 | 1 | 2 | 3 | 4 |
get_priority = [pool_size, local_last_used](size_t i)
/* Consider pool_size equals to 5
* last_used = 1 -> get_priority: 0 1 2 3 4
* last_used = 2 -> get_priority: 4 0 1 2 3
* last_used = 3 -> get_priority: 4 3 0 1 2
* ...
* */
get_priority = [this, pool_size](size_t i)
{
size_t priority = pool_size - 1;
if (i < local_last_used)
priority = pool_size - 1 - (local_last_used - i);
if (i > local_last_used)
priority = i - local_last_used - 1;
return Priority{static_cast<Int64>(priority)};
++i; // To make `i` indexing start with 1 instead of 0 as `last_used` does
return Priority{static_cast<Int64>(i < last_used ? pool_size - i : i - last_used)};
};
break;
}

View File

@ -8,12 +8,7 @@ namespace DB
class GetPriorityForLoadBalancing
{
public:
using Func = std::function<Priority(size_t index)>;
explicit GetPriorityForLoadBalancing(LoadBalancing load_balancing_, size_t last_used_ = 0)
: load_balancing(load_balancing_), last_used(last_used_)
{
}
explicit GetPriorityForLoadBalancing(LoadBalancing load_balancing_) : load_balancing(load_balancing_) {}
GetPriorityForLoadBalancing() = default;
bool operator == (const GetPriorityForLoadBalancing & other) const
@ -28,7 +23,7 @@ public:
return !(*this == other);
}
Func getPriorityFunc(LoadBalancing load_balance, size_t offset, size_t pool_size) const;
std::function<Priority(size_t index)> getPriorityFunc(LoadBalancing load_balance, size_t offset, size_t pool_size) const;
std::vector<size_t> hostname_prefix_distance; /// Prefix distances from name of this host to the names of hosts of pools.
std::vector<size_t> hostname_levenshtein_distance; /// Levenshtein Distances from name of this host to the names of hosts of pools.

View File

@ -124,9 +124,7 @@ public:
size_t max_ignored_errors,
bool fallback_to_stale_replicas,
const TryGetEntryFunc & try_get_entry,
const GetPriorityFunc & get_priority);
size_t getPoolSize() const { return nested_pools.size(); }
const GetPriorityFunc & get_priority = GetPriorityFunc());
protected:
@ -149,7 +147,7 @@ protected:
return std::make_tuple(shared_pool_states, nested_pools, last_error_decrease_time);
}
const NestedPools nested_pools;
NestedPools nested_pools;
const time_t decrease_error_period;
const size_t max_error_cap;

View File

@ -1,9 +1,11 @@
#include "Common/assert_cast.h"
#include "Columns/ColumnNullable.h"
#include "Columns/ColumnsDateTime.h"
#include "Core/DecimalFunctions.h"
#include "DataTypes/IDataType.h"
#include "base/types.h"
#include <Core/MySQL/MySQLUtils.h>
#include <Common/assert_cast.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnsDateTime.h>
#include <Core/DecimalFunctions.h>
#include <DataTypes/IDataType.h>
#include <base/types.h>
namespace DB
{

View File

@ -1,7 +1,7 @@
#pragma once
#include "Core/DecimalFunctions.h"
#include "DataTypes/IDataType.h"
#include <Core/DecimalFunctions.h>
#include <DataTypes/IDataType.h>
namespace DB
{

View File

@ -1,19 +1,20 @@
#include <Core/MySQL/PacketsProtocolBinary.h>
#include <base/DayNum.h>
#include <base/types.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnVector.h>
#include <Columns/IColumn.h>
#include <Common/LocalDate.h>
#include <Common/LocalDateTime.h>
#include <Core/DecimalFunctions.h>
#include <Core/MySQL/IMySQLReadPacket.h>
#include <Core/MySQL/IMySQLWritePacket.h>
#include <Core/MySQL/PacketsProtocolBinary.h>
#include "Common/LocalDate.h"
#include "Common/LocalDateTime.h"
#include "Columns/ColumnNullable.h"
#include "Columns/ColumnVector.h"
#include "Core/DecimalFunctions.h"
#include "DataTypes/DataTypeDateTime64.h"
#include "DataTypes/DataTypeLowCardinality.h"
#include "Formats/FormatSettings.h"
#include "IO/WriteBufferFromString.h"
#include "MySQLUtils.h"
#include "base/DayNum.h"
#include "base/types.h"
#include <Core/MySQL/MySQLUtils.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Formats/FormatSettings.h>
#include <IO/WriteBufferFromString.h>
namespace DB
{

View File

@ -2,11 +2,11 @@
#include <vector>
#include <Columns/IColumn.h>
#include <Core/DecimalFunctions.h>
#include <Core/MySQL/IMySQLReadPacket.h>
#include <Core/MySQL/IMySQLWritePacket.h>
#include "Core/DecimalFunctions.h"
#include "DataTypes/IDataType.h"
#include "DataTypes/Serializations/ISerialization.h"
#include <DataTypes/IDataType.h>
#include <DataTypes/Serializations/ISerialization.h>
namespace DB
{

View File

@ -1,14 +1,15 @@
#include <Core/MySQL/PacketsProtocolText.h>
#include <Core/MySQL/MySQLUtils.h>
#include <Columns/ColumnNullable.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include "Common/assert_cast.h"
#include "Core/MySQL/IMySQLWritePacket.h"
#include "DataTypes/DataTypeLowCardinality.h"
#include "DataTypes/DataTypesDecimal.h"
#include <Common/assert_cast.h>
#include <Core/MySQL/IMySQLWritePacket.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypesDecimal.h>
#include "MySQLUtils.h"
namespace DB
{

View File

@ -281,7 +281,15 @@ public:
/// This allows to receive more signals if failure happens inside onFault function.
/// Example: segfault while symbolizing stack trace.
std::thread([=, this] { onFault(sig, info, context, stack_trace, thread_frame_pointers, thread_num, thread_ptr); }).detach();
try
{
std::thread([=, this] { onFault(sig, info, context, stack_trace, thread_frame_pointers, thread_num, thread_ptr); }).detach();
}
catch (...)
{
/// Likely cannot allocate thread
onFault(sig, info, context, stack_trace, thread_frame_pointers, thread_num, thread_ptr);
}
}
}
}

View File

@ -48,6 +48,7 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile(
const String & source_file_path_,
const FileCache::Key & cache_key_,
FileCachePtr cache_,
const FileCacheUserInfo & user_,
ImplementationBufferCreator implementation_buffer_creator_,
const ReadSettings & settings_,
const String & query_id_,
@ -70,6 +71,7 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile(
, implementation_buffer_creator(implementation_buffer_creator_)
, query_id(query_id_)
, current_buffer_id(getRandomASCIIString(8))
, user(user_)
, allow_seeks_after_first_read(allow_seeks_after_first_read_)
, use_external_buffer(use_external_buffer_)
, query_context_holder(cache_->getQueryContextHolder(query_id, settings_))
@ -127,12 +129,12 @@ bool CachedOnDiskReadBufferFromFile::nextFileSegmentsBatch()
if (settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache)
{
file_segments = cache->get(cache_key, file_offset_of_buffer_end, size, settings.filesystem_cache_segments_batch_size);
file_segments = cache->get(cache_key, file_offset_of_buffer_end, size, settings.filesystem_cache_segments_batch_size, user.user_id);
}
else
{
CreateFileSegmentSettings create_settings(FileSegmentKind::Regular);
file_segments = cache->getOrSet(cache_key, file_offset_of_buffer_end, size, file_size.value(), create_settings, settings.filesystem_cache_segments_batch_size);
file_segments = cache->getOrSet(cache_key, file_offset_of_buffer_end, size, file_size.value(), create_settings, settings.filesystem_cache_segments_batch_size, user);
}
return !file_segments->empty();
}
@ -166,7 +168,7 @@ CachedOnDiskReadBufferFromFile::getCacheReadBuffer(const FileSegment & file_segm
{
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::CachedReadBufferCreateBufferMicroseconds);
auto path = file_segment.getPathInLocalCache();
auto path = file_segment.getPath();
if (cache_file_reader)
{
chassert(cache_file_reader->getFileName() == path);

View File

@ -1,12 +1,15 @@
#pragma once
#include <Interpreters/Cache/FileCache.h>
#include <Interpreters/Cache/FileCacheKey.h>
#include <Interpreters/Cache/FileCache_fwd.h>
#include <Interpreters/Cache/QueryLimit.h>
#include <IO/SeekableReadBuffer.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/ReadSettings.h>
#include <IO/ReadBufferFromFileBase.h>
#include <Interpreters/FilesystemCacheLog.h>
#include <Interpreters/Cache/FileSegment.h>
#include <Interpreters/Cache/UserInfo.h>
namespace CurrentMetrics
@ -24,8 +27,9 @@ public:
CachedOnDiskReadBufferFromFile(
const String & source_file_path_,
const FileCache::Key & cache_key_,
const FileCacheKey & cache_key_,
FileCachePtr cache_,
const FileCacheUserInfo & user_,
ImplementationBufferCreator implementation_buffer_creator_,
const ReadSettings & settings_,
const String & query_id_,
@ -102,7 +106,7 @@ private:
bool nextFileSegmentsBatch();
Poco::Logger * log;
FileCache::Key cache_key;
FileCacheKey cache_key;
String source_file_path;
FileCachePtr cache;
@ -145,13 +149,14 @@ private:
String query_id;
String current_buffer_id;
FileCacheUserInfo user;
bool allow_seeks_after_first_read;
[[maybe_unused]]bool use_external_buffer;
CurrentMetrics::Increment metric_increment{CurrentMetrics::FilesystemCacheReadBuffers};
ProfileEvents::Counters current_file_segment_counters;
FileCache::QueryContextHolderPtr query_context_holder;
FileCacheQueryLimit::QueryContextHolderPtr query_context_holder;
std::shared_ptr<FilesystemCacheLog> cache_log;
};

View File

@ -1,10 +1,10 @@
#include "CachedOnDiskWriteBufferFromFile.h"
#include <Interpreters/Cache/FileCacheFactory.h>
#include <Interpreters/Cache/FileSegment.h>
#include <Common/logger_useful.h>
#include <Interpreters/Cache/FileCacheFactory.h>
#include <Interpreters/Cache/FileCache.h>
#include <Interpreters/Cache/FileSegment.h>
#include <Interpreters/FilesystemCacheLog.h>
#include <Interpreters/Context.h>
#include <IO/SwapHelper.h>
@ -25,6 +25,7 @@ namespace ErrorCodes
FileSegmentRangeWriter::FileSegmentRangeWriter(
FileCache * cache_,
const FileSegment::Key & key_,
const FileCacheUserInfo & user_,
std::shared_ptr<FilesystemCacheLog> cache_log_,
const String & query_id_,
const String & source_path_)
@ -34,6 +35,7 @@ FileSegmentRangeWriter::FileSegmentRangeWriter(
, cache_log(cache_log_)
, query_id(query_id_)
, source_path(source_path_)
, user(user_)
{
}
@ -148,7 +150,7 @@ FileSegment & FileSegmentRangeWriter::allocateFileSegment(size_t offset, FileSeg
/// We set max_file_segment_size to be downloaded,
/// if we have less size to write, file segment will be resized in complete() method.
file_segments = cache->set(key, offset, cache->getMaxFileSegmentSize(), create_settings);
file_segments = cache->set(key, offset, cache->getMaxFileSegmentSize(), create_settings, user);
chassert(file_segments->size() == 1);
return file_segments->front();
}
@ -193,7 +195,6 @@ void FileSegmentRangeWriter::completeFileSegment()
appendFilesystemCacheLog(file_segment);
}
CachedOnDiskWriteBufferFromFile::CachedOnDiskWriteBufferFromFile(
std::unique_ptr<WriteBuffer> impl_,
FileCachePtr cache_,
@ -201,6 +202,7 @@ CachedOnDiskWriteBufferFromFile::CachedOnDiskWriteBufferFromFile(
const FileCache::Key & key_,
const String & query_id_,
const WriteSettings & settings_,
const FileCacheUserInfo & user_,
std::shared_ptr<FilesystemCacheLog> cache_log_)
: WriteBufferFromFileDecorator(std::move(impl_))
, log(&Poco::Logger::get("CachedOnDiskWriteBufferFromFile"))
@ -208,6 +210,7 @@ CachedOnDiskWriteBufferFromFile::CachedOnDiskWriteBufferFromFile(
, source_path(source_path_)
, key(key_)
, query_id(query_id_)
, user(user_)
, throw_on_error_from_cache(settings_.throw_on_error_from_cache)
, cache_log(!query_id_.empty() && settings_.enable_filesystem_cache_log ? cache_log_ : nullptr)
{
@ -233,7 +236,7 @@ void CachedOnDiskWriteBufferFromFile::nextImpl()
{
/// If something was already written to cache, remove it.
cache_writer.reset();
cache->removeKeyIfExists(key);
cache->removeKeyIfExists(key, user.user_id);
throw;
}
@ -246,7 +249,7 @@ void CachedOnDiskWriteBufferFromFile::cacheData(char * data, size_t size, bool t
if (!cache_writer)
{
cache_writer = std::make_unique<FileSegmentRangeWriter>(cache.get(), key, cache_log, query_id, source_path);
cache_writer = std::make_unique<FileSegmentRangeWriter>(cache.get(), key, user, cache_log, query_id, source_path);
}
Stopwatch watch(CLOCK_MONOTONIC);

View File

@ -2,7 +2,9 @@
#include <IO/WriteBufferFromFileDecorator.h>
#include <IO/WriteSettings.h>
#include <Interpreters/Cache/FileCache.h>
#include <Interpreters/Cache/FileCache_fwd.h>
#include <Interpreters/Cache/FileCacheKey.h>
#include <Interpreters/Cache/FileSegment.h>
#include <Interpreters/FilesystemCacheLog.h>
namespace Poco
@ -25,8 +27,12 @@ class FileSegmentRangeWriter
{
public:
FileSegmentRangeWriter(
FileCache * cache_, const FileSegment::Key & key_,
std::shared_ptr<FilesystemCacheLog> cache_log_, const String & query_id_, const String & source_path_);
FileCache * cache_,
const FileSegment::Key & key_,
const FileCacheUserInfo & user_,
std::shared_ptr<FilesystemCacheLog> cache_log_,
const String & query_id_,
const String & source_path_);
/**
* Write a range of file segments. Allocate file segment of `max_file_segment_size` and write to
@ -52,6 +58,7 @@ private:
std::shared_ptr<FilesystemCacheLog> cache_log;
String query_id;
String source_path;
FileCacheUserInfo user;
FileSegmentsHolderPtr file_segments;
@ -71,15 +78,18 @@ public:
std::unique_ptr<WriteBuffer> impl_,
FileCachePtr cache_,
const String & source_path_,
const FileCache::Key & key_,
const FileCacheKey & key_,
const String & query_id_,
const WriteSettings & settings_,
const FileCacheUserInfo & user_,
std::shared_ptr<FilesystemCacheLog> cache_log_);
void nextImpl() override;
void finalizeImpl() override;
bool cachingStopped() const { return cache_in_error_state_or_disabled; }
private:
void cacheData(char * data, size_t size, bool throw_on_error);
@ -87,10 +97,11 @@ private:
FileCachePtr cache;
String source_path;
FileCache::Key key;
FileCacheKey key;
size_t current_download_offset = 0;
const String query_id;
const FileCacheUserInfo user;
bool throw_on_error_from_cache;
bool cache_in_error_state_or_disabled = false;

View File

@ -82,6 +82,7 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c
object_path,
cache_key,
settings.remote_fs_cache,
FileCache::getCommonUser(),
std::move(current_read_buffer_creator),
settings,
query_id,

View File

@ -109,8 +109,6 @@ std::unique_ptr<T> getAzureBlobStorageClientWithAuth(
std::string connection_str;
if (config.has(config_prefix + ".connection_string"))
connection_str = config.getString(config_prefix + ".connection_string");
else if (config.has(config_prefix + ".endpoint"))
connection_str = config.getString(config_prefix + ".endpoint");
if (!connection_str.empty())
return getClientWithConnectionString<T>(connection_str, container_name);
@ -134,14 +132,15 @@ std::unique_ptr<BlobContainerClient> getAzureBlobContainerClient(
{
auto endpoint = processAzureBlobStorageEndpoint(config, config_prefix);
auto container_name = endpoint.container_name;
auto final_url = endpoint.storage_account_url
+ (endpoint.storage_account_url.back() == '/' ? "" : "/")
+ container_name;
auto final_url = container_name.empty()
? endpoint.storage_account_url
: (std::filesystem::path(endpoint.storage_account_url) / container_name).string();
if (endpoint.container_already_exists.value_or(false))
return getAzureBlobStorageClientWithAuth<BlobContainerClient>(final_url, container_name, config, config_prefix);
auto blob_service_client = getAzureBlobStorageClientWithAuth<BlobServiceClient>(endpoint.storage_account_url, container_name, config, config_prefix);
auto blob_service_client = getAzureBlobStorageClientWithAuth<BlobServiceClient>(
endpoint.storage_account_url, container_name, config, config_prefix);
try
{

View File

@ -113,6 +113,7 @@ std::unique_ptr<WriteBufferFromFileBase> CachedObjectStorage::writeObject( /// N
key,
CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() ? std::string(CurrentThread::getQueryId()) : "",
modified_write_settings,
FileCache::getCommonUser(),
Context::getGlobalContextInstance()->getFilesystemCacheLog());
}
@ -125,7 +126,7 @@ void CachedObjectStorage::removeCacheIfExists(const std::string & path_key_for_c
return;
/// Add try catch?
cache->removeKeyIfExists(getCacheKey(path_key_for_cache));
cache->removeKeyIfExists(getCacheKey(path_key_for_cache), FileCache::getCommonUser().user_id);
}
void CachedObjectStorage::removeObject(const StoredObject & object)

View File

@ -538,7 +538,7 @@ struct CopyFileObjectStorageOperation final : public IDiskObjectStorageOperation
for (const auto & object_from : source_blobs)
{
auto object_key = object_storage.generateObjectKeyForPath(to_path);
auto object_key = destination_object_storage.generateObjectKeyForPath(to_path);
auto object_to = StoredObject(object_key.serialize());
object_storage.copyObjectToAnotherObjectStorage(object_from, object_to,read_settings,write_settings, destination_object_storage);

View File

@ -109,13 +109,7 @@ DirectoryIteratorPtr MetadataStorageFromStaticFilesWebServer::iterateDirectory(c
if (!exists(path))
return std::make_unique<StaticDirectoryIterator>(std::move(dir_file_paths));
std::shared_lock shared_lock(object_storage.metadata_mutex);
for (const auto & [file_path, _] : object_storage.files)
{
if (fs::path(parentPath(file_path)) / "" == fs::path(path) / "")
dir_file_paths.emplace_back(file_path);
}
dir_file_paths = object_storage.listDirectory(path);
LOG_TRACE(object_storage.log, "Iterate directory {} with {} files", path, dir_file_paths.size());
return std::make_unique<StaticDirectoryIterator>(std::move(dir_file_paths));
}

View File

@ -31,9 +31,9 @@ namespace ErrorCodes
extern const int FILE_DOESNT_EXIST;
}
void WebObjectStorage::initialize(const String & uri_path, const std::unique_lock<std::shared_mutex> & lock) const
std::vector<fs::path> WebObjectStorage::loadFiles(const String & uri_path, const std::unique_lock<std::shared_mutex> &) const
{
std::vector<String> directories_to_load;
std::vector<fs::path> loaded_files;
LOG_TRACE(log, "Loading metadata for directory: {}", uri_path);
try
@ -74,15 +74,12 @@ void WebObjectStorage::initialize(const String & uri_path, const std::unique_loc
file_data.type = is_directory ? FileType::Directory : FileType::File;
String file_path = fs::path(uri_path) / file_name;
if (file_data.type == FileType::Directory)
{
directories_to_load.push_back(file_path);
}
file_path = file_path.substr(url.size());
LOG_TRACE(&Poco::Logger::get("DiskWeb"), "Adding file: {}, size: {}", file_path, file_data.size);
files.emplace(std::make_pair(file_path, file_data));
loaded_files.emplace_back(file_path);
}
files.emplace(std::make_pair(dir_name, FileData({ .type = FileType::Directory })));
@ -91,7 +88,7 @@ void WebObjectStorage::initialize(const String & uri_path, const std::unique_loc
{
/// 404 - no files
if (e.getHTTPStatus() == Poco::Net::HTTPResponse::HTTP_NOT_FOUND)
return;
return loaded_files;
e.addMessage("while loading disk metadata");
throw;
@ -102,8 +99,7 @@ void WebObjectStorage::initialize(const String & uri_path, const std::unique_loc
throw;
}
for (const auto & directory_path : directories_to_load)
initialize(directory_path, lock);
return loaded_files;
}
@ -135,6 +131,34 @@ WebObjectStorage::FileData WebObjectStorage::getFileInfo(const String & path) co
return file_info.value();
}
std::vector<std::filesystem::path> WebObjectStorage::listDirectory(const String & path) const
{
auto file_info = tryGetFileInfo(path);
if (!file_info)
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "No such file: {}", path);
if (file_info->type != FileType::Directory)
throw Exception(ErrorCodes::LOGICAL_ERROR, "File {} is not a directory", path);
std::vector<std::filesystem::path> result;
if (!file_info->loaded_children)
{
std::unique_lock unique_lock(metadata_mutex);
result = loadFiles(fs::path(url) / path, unique_lock);
file_info->loaded_children = true;
}
else
{
std::shared_lock shared_lock(metadata_mutex);
for (const auto & [file_path, _] : files)
{
if (fs::path(parentPath(file_path)) / "" == fs::path(path) / "")
result.emplace_back(file_path);
}
}
return result;
}
std::optional<WebObjectStorage::FileData> WebObjectStorage::tryGetFileInfo(const String & path) const
{
std::shared_lock shared_lock(metadata_mutex);
@ -142,17 +166,32 @@ std::optional<WebObjectStorage::FileData> WebObjectStorage::tryGetFileInfo(const
if (files.find(path) == files.end())
{
shared_lock.unlock();
std::unique_lock unique_lock(metadata_mutex);
if (files.find(path) == files.end())
{
fs::path index_file_dir = fs::path(url) / path;
if (index_file_dir.has_extension())
index_file_dir = index_file_dir.parent_path();
initialize(index_file_dir, unique_lock);
bool is_file = fs::path(path).has_extension();
if (is_file)
{
const auto parent_path = fs::path(path).parent_path();
auto parent_info = tryGetFileInfo(parent_path);
if (!parent_info)
return std::nullopt; /// Even parent path does not exist.
if (parent_info->loaded_children)
{
return std::nullopt;
}
else
{
std::unique_lock unique_lock(metadata_mutex);
loadFiles(fs::path(url) / parent_path, unique_lock);
parent_info->loaded_children = true;
}
}
/// Files are never deleted from `files` as disk is read only, so no worry that we unlock now.
unique_lock.unlock();
else
{
std::unique_lock unique_lock(metadata_mutex);
loadFiles(fs::path(url) / path, unique_lock);
}
shared_lock.lock();
}

View File

@ -114,6 +114,7 @@ protected:
{
FileType type{};
size_t size = 0;
bool loaded_children = false;
};
using Files = std::map<String, FileData>; /// file path -> file data
@ -121,10 +122,11 @@ protected:
mutable std::shared_mutex metadata_mutex;
std::optional<FileData> tryGetFileInfo(const String & path) const;
std::vector<std::filesystem::path> listDirectory(const String & path) const;
FileData getFileInfo(const String & path) const;
private:
void initialize(const String & path, const std::unique_lock<std::shared_mutex> &) const;
std::vector<std::filesystem::path> loadFiles(const String & path, const std::unique_lock<std::shared_mutex> &) const;
const String url;
Poco::Logger * log;

View File

@ -59,7 +59,6 @@
#include <Common/HashTable/HashMap.h>
#include <DataTypes/DataTypeIPv4andIPv6.h>
#include <Common/IPv6ToBinary.h>
#include "DataTypes/IDataType.h"
#include <Core/Types.h>

View File

@ -27,6 +27,7 @@
#include <base/sleep.h>
#include <algorithm>
namespace ProfileEvents
{
@ -47,6 +48,7 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int TOO_MANY_REDIRECTS;
extern const int BAD_ARGUMENTS;
}
namespace S3
@ -104,6 +106,19 @@ void verifyClientConfiguration(const Aws::Client::ClientConfiguration & client_c
assert_cast<const Client::RetryStrategy &>(*client_config.retryStrategy);
}
void validateCredentials(const Aws::Auth::AWSCredentials& auth_credentials)
{
if (auth_credentials.GetAWSAccessKeyId().empty())
{
return;
}
/// Follow https://docs.aws.amazon.com/IAM/latest/APIReference/API_AccessKey.html
if (!std::all_of(auth_credentials.GetAWSAccessKeyId().begin(), auth_credentials.GetAWSAccessKeyId().end(), isWordCharASCII))
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Access key id has an invalid character");
}
}
void addAdditionalAMZHeadersToCanonicalHeadersList(
Aws::AmazonWebServiceRequest & request,
const HTTPHeaderEntries & extra_headers
@ -129,6 +144,7 @@ std::unique_ptr<Client> Client::create(
const ClientSettings & client_settings)
{
verifyClientConfiguration(client_configuration);
validateCredentials(credentials_provider->GetAWSCredentials());
return std::unique_ptr<Client>(
new Client(max_redirects_, std::move(sse_kms_config_), credentials_provider, client_configuration, sign_payloads, client_settings));
}

View File

@ -13,6 +13,7 @@
#include <base/hex.h>
#include <Common/ThreadPool.h>
#include <Common/ElapsedTimeProfileEventIncrement.h>
#include <Core/ServerUUID.h>
#include <filesystem>
@ -28,29 +29,35 @@ namespace ProfileEvents
extern const Event FilesystemCacheGetMicroseconds;
}
namespace
{
size_t roundDownToMultiple(size_t num, size_t multiple)
{
return (num / multiple) * multiple;
}
size_t roundUpToMultiple(size_t num, size_t multiple)
{
return roundDownToMultiple(num + multiple - 1, multiple);
}
}
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
}
namespace
{
size_t roundDownToMultiple(size_t num, size_t multiple)
{
return (num / multiple) * multiple;
}
size_t roundUpToMultiple(size_t num, size_t multiple)
{
return roundDownToMultiple(num + multiple - 1, multiple);
}
std::string getCommonUserID()
{
auto user_from_context = DB::Context::getGlobalContextInstance()->getFilesystemCacheUser();
const auto user = user_from_context.empty() ? toString(ServerUUID::get()) : user_from_context;
return user;
}
}
void FileCacheReserveStat::update(size_t size, FileSegmentKind kind, bool releasable)
{
auto & local_stat = stat_by_kind[kind];
@ -77,8 +84,9 @@ FileCache::FileCache(const std::string & cache_name, const FileCacheSettings & s
, bypass_cache_threshold(settings.enable_bypass_cache_with_threshold ? settings.bypass_cache_threshold : 0)
, boundary_alignment(settings.boundary_alignment)
, load_metadata_threads(settings.load_metadata_threads)
, write_cache_per_user_directory(settings.write_cache_per_user_id_directory)
, log(&Poco::Logger::get("FileCache(" + cache_name + ")"))
, metadata(settings.base_path, settings.background_download_queue_size_limit, settings.background_download_threads)
, metadata(settings.base_path, settings.background_download_queue_size_limit, settings.background_download_threads, write_cache_per_user_directory)
{
if (settings.cache_policy == "LRU")
main_priority = std::make_unique<LRUFileCachePriority>(settings.max_size, settings.max_elements);
@ -101,19 +109,31 @@ FileCache::Key FileCache::createKeyForPath(const String & path)
return Key(path);
}
const FileCache::UserInfo & FileCache::getCommonUser()
{
static UserInfo user(getCommonUserID(), 0);
return user;
}
const FileCache::UserInfo & FileCache::getInternalUser()
{
static UserInfo user("internal");
return user;
}
const String & FileCache::getBasePath() const
{
return metadata.getBaseDirectory();
}
String FileCache::getPathInLocalCache(const Key & key, size_t offset, FileSegmentKind segment_kind) const
String FileCache::getFileSegmentPath(const Key & key, size_t offset, FileSegmentKind segment_kind, const UserInfo & user) const
{
return metadata.getPathForFileSegment(key, offset, segment_kind);
return metadata.getFileSegmentPath(key, offset, segment_kind, user);
}
String FileCache::getPathInLocalCache(const Key & key) const
String FileCache::getKeyPath(const Key & key, const UserInfo & user) const
{
return metadata.getPathForKey(key);
return metadata.getKeyPath(key, user);
}
void FileCache::assertInitialized() const
@ -276,9 +296,10 @@ std::vector<FileSegment::Range> FileCache::splitRange(size_t offset, size_t size
size_t remaining_size = size;
FileSegments file_segments;
const size_t max_size = max_file_segment_size.load();
while (current_pos < end_pos_non_included)
{
auto current_file_segment_size = std::min(remaining_size, max_file_segment_size);
auto current_file_segment_size = std::min(remaining_size, max_size);
ranges.emplace_back(current_pos, current_pos + current_file_segment_size - 1);
remaining_size -= current_file_segment_size;
@ -305,9 +326,10 @@ FileSegments FileCache::splitRangeIntoFileSegments(
size_t remaining_size = size;
FileSegments file_segments;
const size_t max_size = max_file_segment_size.load();
while (current_pos < end_pos_non_included && (!file_segments_limit || file_segments.size() < file_segments_limit))
{
current_file_segment_size = std::min(remaining_size, max_file_segment_size);
current_file_segment_size = std::min(remaining_size, max_size);
remaining_size -= current_file_segment_size;
auto file_segment_metadata_it = addFileSegment(
@ -468,11 +490,12 @@ FileSegmentsHolderPtr FileCache::set(
const Key & key,
size_t offset,
size_t size,
const CreateFileSegmentSettings & create_settings)
const CreateFileSegmentSettings & create_settings,
const UserInfo & user)
{
assertInitialized();
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY);
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY, user);
FileSegment::Range range(offset, offset + size - 1);
auto file_segments = getImpl(*locked_key, range, /* file_segments_limit */0);
@ -502,7 +525,8 @@ FileCache::getOrSet(
size_t size,
size_t file_size,
const CreateFileSegmentSettings & create_settings,
size_t file_segments_limit)
size_t file_segments_limit,
const UserInfo & user)
{
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::FilesystemCacheGetOrSetMicroseconds);
@ -516,7 +540,7 @@ FileCache::getOrSet(
chassert(aligned_offset <= range.left);
chassert(aligned_end_offset >= range.right);
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY);
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY, user);
/// Get all segments which intersect with the given range.
auto file_segments = getImpl(*locked_key, range, file_segments_limit);
@ -631,13 +655,18 @@ FileCache::getOrSet(
return std::make_unique<FileSegmentsHolder>(std::move(file_segments));
}
FileSegmentsHolderPtr FileCache::get(const Key & key, size_t offset, size_t size, size_t file_segments_limit)
FileSegmentsHolderPtr FileCache::get(
const Key & key,
size_t offset,
size_t size,
size_t file_segments_limit,
const UserID & user_id)
{
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::FilesystemCacheGetMicroseconds);
assertInitialized();
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::RETURN_NULL);
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::RETURN_NULL, UserInfo(user_id));
if (locked_key)
{
FileSegment::Range range(offset, offset + size - 1);
@ -705,7 +734,7 @@ KeyMetadata::iterator FileCache::addFileSegment(
auto & stash_records = stash->records;
stash_records.emplace(
stash_key, stash->queue->add(locked_key.getKeyMetadata(), offset, 0, *lock));
stash_key, stash->queue->add(locked_key.getKeyMetadata(), offset, 0, locked_key.getKeyMetadata()->user, *lock));
if (stash->queue->getElementsCount(*lock) > stash->queue->getElementsLimit(*lock))
stash->queue->pop(*lock);
@ -738,7 +767,11 @@ KeyMetadata::iterator FileCache::addFileSegment(
return file_segment_metadata_it;
}
bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCacheReserveStat & reserve_stat)
bool FileCache::tryReserve(
FileSegment & file_segment,
const size_t size,
FileCacheReserveStat & reserve_stat,
const UserInfo & user)
{
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::FilesystemCacheReserveMicroseconds);
@ -780,7 +813,8 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa
if (query_priority)
{
if (!query_priority->collectCandidatesForEviction(size, reserve_stat, eviction_candidates, {}, finalize_eviction_func, cache_lock))
if (!query_priority->collectCandidatesForEviction(
size, reserve_stat, eviction_candidates, {}, finalize_eviction_func, user.user_id, cache_lock))
return false;
LOG_TEST(log, "Query limits satisfied (while reserving for {}:{})", file_segment.key(), file_segment.offset());
@ -793,7 +827,8 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa
auto queue_iterator = file_segment.getQueueIterator();
chassert(!queue_iterator || file_segment.getReservedSize() > 0);
if (!main_priority->collectCandidatesForEviction(size, reserve_stat, eviction_candidates, queue_iterator, finalize_eviction_func, cache_lock))
if (!main_priority->collectCandidatesForEviction(
size, reserve_stat, eviction_candidates, queue_iterator, finalize_eviction_func, user.user_id, cache_lock))
return false;
if (!file_segment.getKeyMetadata()->createBaseDirectory())
@ -812,7 +847,7 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa
{
/// Space reservation is incremental, so file_segment_metadata is created first (with state empty),
/// and getQueueIterator() is assigned on first space reservation attempt.
queue_iterator = main_priority->add(file_segment.getKeyMetadata(), file_segment.offset(), size, cache_lock);
queue_iterator = main_priority->add(file_segment.getKeyMetadata(), file_segment.offset(), size, user, cache_lock);
file_segment.setQueueIterator(queue_iterator);
}
@ -825,7 +860,7 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa
if (query_queue_it)
query_queue_it->updateSize(size);
else
query_context->add(file_segment.getKeyMetadata(), file_segment.offset(), size, cache_lock);
query_context->add(file_segment.getKeyMetadata(), file_segment.offset(), size, user, cache_lock);
}
if (main_priority->getSize(cache_lock) > (1ull << 63))
@ -834,40 +869,40 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa
return true;
}
void FileCache::iterate(IterateFunc && func)
void FileCache::iterate(IterateFunc && func, const UserID & user_id)
{
return metadata.iterate([&](const LockedKey & locked_key)
{
for (const auto & file_segment_metadata : locked_key)
func(FileSegment::getInfo(file_segment_metadata.second->file_segment));
});
}, user_id);
}
void FileCache::removeKey(const Key & key)
void FileCache::removeKey(const Key & key, const UserID & user_id)
{
assertInitialized();
metadata.removeKey(key, /* if_exists */false, /* if_releasable */true);
metadata.removeKey(key, /* if_exists */false, /* if_releasable */true, user_id);
}
void FileCache::removeKeyIfExists(const Key & key)
void FileCache::removeKeyIfExists(const Key & key, const UserID & user_id)
{
assertInitialized();
metadata.removeKey(key, /* if_exists */true, /* if_releasable */true);
metadata.removeKey(key, /* if_exists */true, /* if_releasable */true, user_id);
}
void FileCache::removeFileSegment(const Key & key, size_t offset)
void FileCache::removeFileSegment(const Key & key, size_t offset, const UserID & user_id)
{
assertInitialized();
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW);
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW, UserInfo(user_id));
locked_key->removeFileSegment(offset);
}
void FileCache::removePathIfExists(const String & path)
void FileCache::removePathIfExists(const String & path, const UserID & user_id)
{
removeKeyIfExists(createKeyForPath(path));
removeKeyIfExists(createKeyForPath(path), user_id);
}
void FileCache::removeAllReleasable()
void FileCache::removeAllReleasable(const UserID & user_id)
{
assertInitialized();
@ -875,7 +910,7 @@ void FileCache::removeAllReleasable()
assertCacheCorrectness();
#endif
metadata.removeAllKeys(/* if_releasable */true);
metadata.removeAllKeys(/* if_releasable */true, user_id);
if (stash)
{
@ -1002,6 +1037,24 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir)
return;
}
UserInfo user;
if (write_cache_per_user_directory)
{
auto filename = keys_dir.filename().string();
auto pos = filename.find_last_of('.');
if (pos == std::string::npos)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected file format: {}", filename);
user = UserInfo(filename.substr(0, pos), parse<UInt64>(filename.substr(pos + 1)));
LOG_TEST(log, "Loading cache for user {}", user.user_id);
}
else
{
user = getCommonUser();
}
UInt64 offset = 0, size = 0;
for (; key_it != fs::directory_iterator(); key_it++)
{
@ -1024,7 +1077,7 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir)
}
const auto key = Key::fromKeyString(key_directory.filename().string());
auto key_metadata = metadata.getKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY, /* is_initial_load */true);
auto key_metadata = metadata.getKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY, user, /* is_initial_load */true);
for (fs::directory_iterator offset_it{key_directory}; offset_it != fs::directory_iterator(); ++offset_it)
{
@ -1072,9 +1125,9 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir)
auto lock = lockCache();
size_limit = main_priority->getSizeLimit(lock);
limits_satisfied = main_priority->canFit(size, lock);
limits_satisfied = main_priority->canFit(size, lock, nullptr, true);
if (limits_satisfied)
cache_it = main_priority->add(key_metadata, offset, size, lock, /* is_startup */true);
cache_it = main_priority->add(key_metadata, offset, size, user, lock, /* best_effort */true);
/// TODO: we can get rid of this lockCache() if we first load everything in parallel
/// without any mutual lock between loading threads, and only after do removeOverflow().
@ -1128,7 +1181,7 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir)
if (key_metadata->sizeUnlocked() == 0)
{
metadata.removeKey(key, false, false);
metadata.removeKey(key, false, false, getInternalUser().user_id);
}
}
}
@ -1147,7 +1200,7 @@ void FileCache::deactivateBackgroundOperations()
metadata.shutdown();
}
std::vector<FileSegment::Info> FileCache::getFileSegmentInfos()
std::vector<FileSegment::Info> FileCache::getFileSegmentInfos(const UserID & user_id)
{
assertInitialized();
#ifndef NDEBUG
@ -1159,20 +1212,20 @@ std::vector<FileSegment::Info> FileCache::getFileSegmentInfos()
{
for (const auto & [_, file_segment_metadata] : locked_key)
file_segments.push_back(FileSegment::getInfo(file_segment_metadata->file_segment));
});
}, user_id);
return file_segments;
}
std::vector<FileSegment::Info> FileCache::getFileSegmentInfos(const Key & key)
std::vector<FileSegment::Info> FileCache::getFileSegmentInfos(const Key & key, const UserID & user_id)
{
std::vector<FileSegment::Info> file_segments;
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW_LOGICAL);
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW_LOGICAL, UserInfo(user_id));
for (const auto & [_, file_segment_metadata] : *locked_key)
file_segments.push_back(FileSegment::getInfo(file_segment_metadata->file_segment));
return file_segments;
}
std::vector<FileSegment::Info> FileCache::dumpQueue()
IFileCachePriority::PriorityDumpPtr FileCache::dumpQueue()
{
assertInitialized();
return main_priority->dump(lockCache());
@ -1182,7 +1235,7 @@ std::vector<String> FileCache::tryGetCachePaths(const Key & key)
{
assertInitialized();
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::RETURN_NULL);
auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::RETURN_NULL, getInternalUser());
if (!locked_key)
return {};
@ -1190,8 +1243,9 @@ std::vector<String> FileCache::tryGetCachePaths(const Key & key)
for (const auto & [offset, file_segment_metadata] : *locked_key)
{
if (file_segment_metadata->file_segment->state() == FileSegment::State::DOWNLOADED)
cache_paths.push_back(metadata.getPathForFileSegment(key, offset, file_segment_metadata->file_segment->getKind()));
const auto & file_segment = *file_segment_metadata->file_segment;
if (file_segment.state() == FileSegment::State::DOWNLOADED)
cache_paths.push_back(locked_key->getKeyMetadata()->getFileSegmentPath(file_segment));
}
return cache_paths;
}
@ -1214,7 +1268,7 @@ void FileCache::assertCacheCorrectness()
{
chassert(file_segment_metadata->file_segment->assertCorrectness());
}
});
}, getInternalUser().user_id);
}
void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, FileCacheSettings & actual_settings)
@ -1286,26 +1340,10 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings,
actual_settings.max_elements = main_priority->getElementsLimit(cache_lock);
}
}
}
FileCache::QueryContextHolder::QueryContextHolder(
const String & query_id_,
FileCache * cache_,
FileCacheQueryLimit::QueryContextPtr context_)
: query_id(query_id_)
, cache(cache_)
, context(context_)
{
}
FileCache::QueryContextHolder::~QueryContextHolder()
{
/// If only the query_map and the current holder hold the context_query,
/// the query has been completed and the query_context is released.
if (context && context.use_count() == 2)
if (new_settings.max_file_segment_size != actual_settings.max_file_segment_size)
{
auto lock = cache->lockCache();
cache->query_limit->removeQueryContext(query_id, lock);
max_file_segment_size = actual_settings.max_file_segment_size = new_settings.max_file_segment_size;
}
}
@ -1317,7 +1355,7 @@ FileCache::QueryContextHolderPtr FileCache::getQueryContextHolder(
auto lock = lockCache();
auto context = query_limit->getOrSetQueryContext(query_id, read_settings, lock);
return std::make_unique<QueryContextHolder>(query_id, this, std::move(context));
return std::make_unique<QueryContextHolder>(query_id, this, query_limit.get(), std::move(context));
}
std::vector<FileSegment::Info> FileCache::sync()
@ -1327,7 +1365,7 @@ std::vector<FileSegment::Info> FileCache::sync()
{
auto broken = locked_key.sync();
file_segments.insert(file_segments.end(), broken.begin(), broken.end());
});
}, getInternalUser().user_id);
return file_segments;
}

View File

@ -17,6 +17,7 @@
#include <Interpreters/Cache/QueryLimit.h>
#include <Interpreters/Cache/FileCache_fwd_internal.h>
#include <Interpreters/Cache/FileCacheSettings.h>
#include <Interpreters/Cache/UserInfo.h>
#include <filesystem>
@ -34,12 +35,29 @@ struct FileCacheReserveStat
size_t non_releasable_size = 0;
size_t non_releasable_count = 0;
Stat & operator +=(const Stat & other)
{
releasable_size += other.releasable_size;
releasable_count += other.releasable_count;
non_releasable_size += other.non_releasable_size;
non_releasable_count += other.non_releasable_count;
return *this;
}
};
Stat stat;
std::unordered_map<FileSegmentKind, Stat> stat_by_kind;
void update(size_t size, FileSegmentKind kind, bool releasable);
FileCacheReserveStat & operator +=(const FileCacheReserveStat & other)
{
stat += other.stat;
for (const auto & [name, stat_] : other.stat_by_kind)
stat_by_kind[name] += stat_;
return *this;
}
};
/// Local cache for remote filesystem files, represented as a set of non-overlapping non-empty file segments.
@ -51,6 +69,9 @@ public:
using QueryLimit = DB::FileCacheQueryLimit;
using Priority = IFileCachePriority;
using PriorityEntry = IFileCachePriority::Entry;
using QueryContextHolder = FileCacheQueryLimit::QueryContextHolder;
using UserInfo = FileCacheUserInfo;
using UserID = UserInfo::UserID;
FileCache(const std::string & cache_name, const FileCacheSettings & settings);
@ -62,9 +83,13 @@ public:
static Key createKeyForPath(const String & path);
String getPathInLocalCache(const Key & key, size_t offset, FileSegmentKind segment_kind) const;
static const UserInfo & getCommonUser();
String getPathInLocalCache(const Key & key) const;
static const UserInfo & getInternalUser();
String getFileSegmentPath(const Key & key, size_t offset, FileSegmentKind segment_kind, const UserInfo & user) const;
String getKeyPath(const Key & key, const UserInfo & user) const;
/**
* Given an `offset` and `size` representing [offset, offset + size) bytes interval,
@ -83,7 +108,8 @@ public:
size_t size,
size_t file_size,
const CreateFileSegmentSettings & settings,
size_t file_segments_limit = 0);
size_t file_segments_limit,
const UserInfo & user);
/**
* Segments in returned list are ordered in ascending order and represent a full contiguous
@ -94,24 +120,34 @@ public:
* with the destruction of the holder, while in getOrSet() EMPTY file segments can eventually change
* it's state (and become DOWNLOADED).
*/
FileSegmentsHolderPtr get(const Key & key, size_t offset, size_t size, size_t file_segments_limit);
FileSegmentsHolderPtr get(
const Key & key,
size_t offset,
size_t size,
size_t file_segments_limit,
const UserID & user_id);
FileSegmentsHolderPtr set(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings);
FileSegmentsHolderPtr set(
const Key & key,
size_t offset,
size_t size,
const CreateFileSegmentSettings & settings,
const UserInfo & user);
/// Remove file segment by `key` and `offset`. Throws if file segment does not exist.
void removeFileSegment(const Key & key, size_t offset);
void removeFileSegment(const Key & key, size_t offset, const UserID & user_id);
/// Remove files by `key`. Throws if key does not exist.
void removeKey(const Key & key);
void removeKey(const Key & key, const UserID & user_id);
/// Remove files by `key`.
void removeKeyIfExists(const Key & key);
void removeKeyIfExists(const Key & key, const UserID & user_id);
/// Removes files by `path`.
void removePathIfExists(const String & path);
void removePathIfExists(const String & path, const UserID & user_id);
/// Remove files by `key`.
void removeAllReleasable();
void removeAllReleasable(const UserID & user_id);
std::vector<String> tryGetCachePaths(const Key & key);
@ -121,48 +157,41 @@ public:
size_t getMaxFileSegmentSize() const { return max_file_segment_size; }
bool tryReserve(FileSegment & file_segment, size_t size, FileCacheReserveStat & stat);
bool tryReserve(
FileSegment & file_segment,
size_t size,
FileCacheReserveStat & stat,
const UserInfo & user);
std::vector<FileSegment::Info> getFileSegmentInfos();
std::vector<FileSegment::Info> getFileSegmentInfos(const UserID & user_id);
std::vector<FileSegment::Info> getFileSegmentInfos(const Key & key);
std::vector<FileSegment::Info> getFileSegmentInfos(const Key & key, const UserID & user_id);
std::vector<FileSegment::Info> dumpQueue();
IFileCachePriority::PriorityDumpPtr dumpQueue();
void deactivateBackgroundOperations();
/// For per query cache limit.
struct QueryContextHolder : private boost::noncopyable
{
QueryContextHolder(const String & query_id_, FileCache * cache_, QueryLimit::QueryContextPtr context_);
QueryContextHolder() = default;
~QueryContextHolder();
String query_id;
FileCache * cache = nullptr;
QueryLimit::QueryContextPtr context;
};
using QueryContextHolderPtr = std::unique_ptr<QueryContextHolder>;
QueryContextHolderPtr getQueryContextHolder(const String & query_id, const ReadSettings & settings);
CacheGuard::Lock lockCache() const;
std::vector<FileSegment::Info> sync();
using QueryContextHolderPtr = std::unique_ptr<QueryContextHolder>;
QueryContextHolderPtr getQueryContextHolder(const String & query_id, const ReadSettings & settings);
using IterateFunc = std::function<void(const FileSegmentInfo &)>;
void iterate(IterateFunc && func);
void iterate(IterateFunc && func, const UserID & user_id);
void applySettingsIfPossible(const FileCacheSettings & new_settings, FileCacheSettings & actual_settings);
private:
using KeyAndOffset = FileCacheKeyAndOffset;
const size_t max_file_segment_size;
std::atomic<size_t> max_file_segment_size;
const size_t bypass_cache_threshold;
const size_t boundary_alignment;
size_t load_metadata_threads;
const bool write_cache_per_user_directory;
Poco::Logger * log;

View File

@ -173,4 +173,10 @@ void FileCacheFactory::updateSettingsFromConfig(const Poco::Util::AbstractConfig
}
}
void FileCacheFactory::clear()
{
std::lock_guard lock(mutex);
caches_by_name.clear();
}
}

View File

@ -55,6 +55,8 @@ public:
void updateSettingsFromConfig(const Poco::Util::AbstractConfiguration & config);
void clear();
private:
std::mutex mutex;
CacheByName caches_by_name;

View File

@ -21,6 +21,7 @@ struct FileCacheKey
static FileCacheKey random();
bool operator==(const FileCacheKey & other) const { return key == other.key; }
bool operator<(const FileCacheKey & other) const { return key < other.key; }
static FileCacheKey fromKeyString(const std::string & key_str);
};

View File

@ -5,6 +5,7 @@
#include <Common/NamedCollections/NamedCollections.h>
#include <boost/algorithm/string/case_conv.hpp>
#include <IO/ReadHelpers.h>
#include <Common/logger_useful.h>
namespace DB
{
@ -74,6 +75,9 @@ void FileCacheSettings::loadImpl(FuncHas has, FuncGetUInt get_uint, FuncGetStrin
if (has("slru_size_ratio"))
slru_size_ratio = get_double("slru_size_ratio");
if (has("write_cache_per_user_id_directory"))
slru_size_ratio = get_uint("write_cache_per_user_id_directory");
}
void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)

View File

@ -33,6 +33,8 @@ struct FileCacheSettings
size_t load_metadata_threads = FILECACHE_DEFAULT_LOAD_METADATA_THREADS;
bool write_cache_per_user_id_directory = false;
std::string cache_policy = "LRU";
double slru_size_ratio = 0.5;

View File

@ -85,7 +85,7 @@ FileSegment::FileSegment(
case (State::DOWNLOADED):
{
reserved_size = downloaded_size = size_;
chassert(fs::file_size(getPathInLocalCache()) == size_);
chassert(fs::file_size(getPath()) == size_);
chassert(queue_iterator);
chassert(key_metadata.lock());
break;
@ -115,12 +115,12 @@ FileSegment::State FileSegment::state() const
return download_state;
}
String FileSegment::getPathInLocalCache() const
String FileSegment::getPath() const
{
return getKeyMetadata()->getFileSegmentPath(*this);
}
String FileSegment::tryGetPathInLocalCache() const
String FileSegment::tryGetPath() const
{
auto metadata = tryGetKeyMetadata();
if (!metadata)
@ -182,7 +182,7 @@ void FileSegment::setDownloadedSize(size_t delta)
{
auto lock = lockFileSegment();
downloaded_size += delta;
assert(downloaded_size == std::filesystem::file_size(getPathInLocalCache()));
assert(downloaded_size == std::filesystem::file_size(getPath()));
}
bool FileSegment::isDownloaded() const
@ -339,7 +339,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset)
if (!size)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Writing zero size is not allowed");
const auto file_segment_path = getPathInLocalCache();
const auto file_segment_path = getPath();
{
auto lock = lockFileSegment();
@ -531,7 +531,7 @@ bool FileSegment::reserve(size_t size_to_reserve, FileCacheReserveStat * reserve
if (!reserve_stat)
reserve_stat = &dummy_stat;
bool reserved = cache->tryReserve(*this, size_to_reserve, *reserve_stat);
bool reserved = cache->tryReserve(*this, size_to_reserve, *reserve_stat, getKeyMetadata()->user);
if (!reserved)
setDownloadFailedUnlocked(lockFileSegment());
@ -554,7 +554,7 @@ void FileSegment::setDownloadedUnlocked(const FileSegmentGuard::Lock &)
}
chassert(downloaded_size > 0);
chassert(fs::file_size(getPathInLocalCache()) == downloaded_size);
chassert(fs::file_size(getPath()) == downloaded_size);
}
void FileSegment::setDownloadFailed()
@ -654,7 +654,7 @@ void FileSegment::complete()
case State::DOWNLOADED:
{
chassert(current_downloaded_size == range().size());
chassert(current_downloaded_size == fs::file_size(getPathInLocalCache()));
chassert(current_downloaded_size == fs::file_size(getPath()));
chassert(!cache_writer);
chassert(!remote_file_reader);
break;
@ -800,7 +800,7 @@ bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock &) cons
chassert(downloaded_size == reserved_size);
chassert(downloaded_size == range().size());
chassert(downloaded_size > 0);
chassert(std::filesystem::file_size(getPathInLocalCache()) > 0);
chassert(std::filesystem::file_size(getPath()) > 0);
chassert(queue_iterator);
check_iterator(queue_iterator);
}
@ -844,10 +844,11 @@ void FileSegment::assertNotDetachedUnlocked(const FileSegmentGuard::Lock & lock)
FileSegment::Info FileSegment::getInfo(const FileSegmentPtr & file_segment)
{
auto lock = file_segment->lockFileSegment();
auto key_metadata = file_segment->tryGetKeyMetadata();
return Info{
.key = file_segment->key(),
.offset = file_segment->offset(),
.path = file_segment->tryGetPathInLocalCache(),
.path = file_segment->tryGetPath(),
.range_left = file_segment->range().left,
.range_right = file_segment->range().right,
.kind = file_segment->segment_kind,
@ -858,6 +859,8 @@ FileSegment::Info FileSegment::getInfo(const FileSegmentPtr & file_segment)
.references = static_cast<uint64_t>(file_segment.use_count()),
.is_unbound = file_segment->is_unbound,
.queue_entry_type = file_segment->queue_iterator ? file_segment->queue_iterator->getType() : QueueEntryType::None,
.user_id = key_metadata->user.user_id,
.user_weight = key_metadata->user.weight.value(),
};
}
@ -890,7 +893,6 @@ void FileSegment::setDetachedState(const FileSegmentGuard::Lock & lock)
{
setDownloadState(State::DETACHED, lock);
key_metadata.reset();
cache = nullptr;
queue_iterator = nullptr;
try
{

View File

@ -112,7 +112,7 @@ public:
bool isUnbound() const { return is_unbound; }
String getPathInLocalCache() const;
String getPath() const;
int getFlagsForLocalRead() const { return O_RDONLY | O_CLOEXEC; }
@ -243,7 +243,7 @@ private:
LockedKeyPtr lockKeyMetadata(bool assert_exists = true) const;
FileSegmentGuard::Lock lockFileSegment() const;
String tryGetPathInLocalCache() const;
String tryGetPath() const;
Key file_key;
Range segment_range;

View File

@ -78,5 +78,7 @@ namespace DB
uint64_t references;
bool is_unbound;
FileCacheQueueEntryType queue_entry_type;
std::string user_id;
uint64_t user_weight;
};
}

View File

@ -7,6 +7,7 @@
#include <Interpreters/Cache/Guards.h>
#include <Interpreters/Cache/IFileCachePriority.h>
#include <Interpreters/Cache/FileCache_fwd_internal.h>
#include <Interpreters/Cache/UserInfo.h>
namespace DB
{
@ -18,6 +19,8 @@ class IFileCachePriority : private boost::noncopyable
public:
using Key = FileCacheKey;
using QueueEntryType = FileCacheQueueEntryType;
using UserInfo = FileCacheUserInfo;
using UserID = UserInfo::UserID;
struct Entry
{
@ -52,8 +55,6 @@ public:
};
using IteratorPtr = std::shared_ptr<Iterator>;
IFileCachePriority(size_t max_size_, size_t max_elements_);
virtual ~IFileCachePriority() = default;
size_t getElementsLimit(const CacheGuard::Lock &) const { return max_elements; }
@ -69,14 +70,28 @@ public:
KeyMetadataPtr key_metadata,
size_t offset,
size_t size,
const UserInfo & user,
const CacheGuard::Lock &,
bool is_startup = false) = 0;
bool best_effort = false) = 0;
virtual bool canFit(size_t size, const CacheGuard::Lock &) const = 0;
/// `reservee` is the entry for which are reserving now.
/// It does not exist, if it is the first space reservation attempt
/// for the corresponding file segment.
virtual bool canFit( /// NOLINT
size_t size,
const CacheGuard::Lock &,
IteratorPtr reservee = nullptr,
bool best_effort = false) const = 0;
virtual void shuffle(const CacheGuard::Lock &) = 0;
virtual std::vector<FileSegmentInfo> dump(const CacheGuard::Lock &) = 0;
struct IPriorityDump
{
virtual ~IPriorityDump() = default;
};
using PriorityDumpPtr = std::shared_ptr<IPriorityDump>;
virtual PriorityDumpPtr dump(const CacheGuard::Lock &) = 0;
using FinalizeEvictionFunc = std::function<void(const CacheGuard::Lock & lk)>;
virtual bool collectCandidatesForEviction(
@ -85,11 +100,14 @@ public:
EvictionCandidates & res,
IFileCachePriority::IteratorPtr reservee,
FinalizeEvictionFunc & finalize_eviction_func,
const UserID & user_id,
const CacheGuard::Lock &) = 0;
virtual bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CacheGuard::Lock &) = 0;
protected:
IFileCachePriority(size_t max_size_, size_t max_elements_);
size_t max_size = 0;
size_t max_elements = 0;
};

View File

@ -29,12 +29,22 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
LRUFileCachePriority::LRUFileCachePriority(size_t max_size_, size_t max_elements_, StatePtr state_)
: IFileCachePriority(max_size_, max_elements_)
{
if (state_)
state = state_;
else
state = std::make_shared<State>();
}
IFileCachePriority::IteratorPtr LRUFileCachePriority::add( /// NOLINT
KeyMetadataPtr key_metadata,
size_t offset,
size_t size,
const UserInfo &,
const CacheGuard::Lock & lock,
bool /* is_startup */)
bool)
{
return std::make_shared<LRUIterator>(add(std::make_shared<Entry>(key_metadata->key, offset, size, key_metadata), lock));
}
@ -63,12 +73,12 @@ LRUFileCachePriority::LRUIterator LRUFileCachePriority::add(EntryPtr entry, cons
#endif
const auto & size_limit = getSizeLimit(lock);
if (size_limit && current_size + entry->size > size_limit)
if (size_limit && state->current_size + entry->size > size_limit)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Not enough space to add {}:{} with size {}: current size: {}/{}",
entry->key, entry->offset, entry->size, current_size, size_limit);
entry->key, entry->offset, entry->size, state->current_size, size_limit);
}
auto iterator = queue.insert(queue.end(), entry);
@ -102,13 +112,13 @@ LRUFileCachePriority::LRUQueue::iterator LRUFileCachePriority::remove(LRUQueue::
void LRUFileCachePriority::updateSize(int64_t size)
{
current_size += size;
state->current_size += size;
CurrentMetrics::add(CurrentMetrics::FilesystemCacheSize, size);
}
void LRUFileCachePriority::updateElementsCount(int64_t num)
{
current_elements_num += num;
state->current_elements_num += num;
CurrentMetrics::add(CurrentMetrics::FilesystemCacheElements, num);
}
@ -189,7 +199,11 @@ void LRUFileCachePriority::iterate(IterateFunc && func, const CacheGuard::Lock &
}
}
bool LRUFileCachePriority::canFit(size_t size, const CacheGuard::Lock & lock) const
bool LRUFileCachePriority::canFit( /// NOLINT
size_t size,
const CacheGuard::Lock & lock,
IteratorPtr,
bool) const
{
return canFit(size, 0, 0, lock);
}
@ -200,8 +214,8 @@ bool LRUFileCachePriority::canFit(
size_t released_elements_assumption,
const CacheGuard::Lock &) const
{
return (max_size == 0 || (current_size + size - released_size_assumption <= max_size))
&& (max_elements == 0 || current_elements_num + 1 - released_elements_assumption <= max_elements);
return (max_size == 0 || (state->current_size + size - released_size_assumption <= max_size))
&& (max_elements == 0 || state->current_elements_num + 1 - released_elements_assumption <= max_elements);
}
bool LRUFileCachePriority::collectCandidatesForEviction(
@ -210,6 +224,7 @@ bool LRUFileCachePriority::collectCandidatesForEviction(
EvictionCandidates & res,
IFileCachePriority::IteratorPtr,
FinalizeEvictionFunc &,
const UserID &,
const CacheGuard::Lock & lock)
{
if (canFit(size, lock))
@ -282,7 +297,7 @@ LRUFileCachePriority::LRUIterator LRUFileCachePriority::move(LRUIterator & it, L
return LRUIterator(this, it.iterator);
}
std::vector<FileSegmentInfo> LRUFileCachePriority::dump(const CacheGuard::Lock & lock)
IFileCachePriority::PriorityDumpPtr LRUFileCachePriority::dump(const CacheGuard::Lock & lock)
{
std::vector<FileSegmentInfo> res;
iterate([&](LockedKey &, const FileSegmentMetadataPtr & segment_metadata)
@ -290,7 +305,7 @@ std::vector<FileSegmentInfo> LRUFileCachePriority::dump(const CacheGuard::Lock &
res.emplace_back(FileSegment::getInfo(segment_metadata->file_segment));
return IterationResult::CONTINUE;
}, lock);
return res;
return std::make_shared<LRUPriorityDump>(res);
}
bool LRUFileCachePriority::modifySizeLimits(
@ -301,8 +316,8 @@ bool LRUFileCachePriority::modifySizeLimits(
auto check_limits_satisfied = [&]()
{
return (max_size_ == 0 || current_size <= max_size_)
&& (max_elements_ == 0 || current_elements_num <= max_elements_);
return (max_size_ == 0 || state->current_size <= max_size_)
&& (max_elements_ == 0 || state->current_elements_num <= max_elements_);
};
if (check_limits_satisfied())

View File

@ -13,26 +13,34 @@ namespace DB
/// the head of the queue, and the record with the highest priority is stored at the tail.
class LRUFileCachePriority final : public IFileCachePriority
{
private:
class LRUIterator;
using LRUQueue = std::list<EntryPtr>;
friend class SLRUFileCachePriority;
protected:
struct State
{
std::atomic<size_t> current_size = 0;
std::atomic<size_t> current_elements_num = 0;
};
using StatePtr = std::shared_ptr<State>;
public:
LRUFileCachePriority(size_t max_size_, size_t max_elements_) : IFileCachePriority(max_size_, max_elements_) {}
LRUFileCachePriority(size_t max_size_, size_t max_elements_, StatePtr state_ = nullptr);
size_t getSize(const CacheGuard::Lock &) const override { return current_size; }
size_t getSize(const CacheGuard::Lock &) const override { return state->current_size; }
size_t getElementsCount(const CacheGuard::Lock &) const override { return current_elements_num; }
size_t getElementsCount(const CacheGuard::Lock &) const override { return state->current_elements_num; }
bool canFit(size_t size, const CacheGuard::Lock &) const override;
bool canFit( /// NOLINT
size_t size,
const CacheGuard::Lock &,
IteratorPtr reservee = nullptr,
bool best_effort = false) const override;
IteratorPtr add( /// NOLINT
KeyMetadataPtr key_metadata,
size_t offset,
size_t size,
const UserInfo & user,
const CacheGuard::Lock &,
bool is_startup = false) override;
bool best_effort = false) override;
bool collectCandidatesForEviction(
size_t size,
@ -40,27 +48,34 @@ public:
EvictionCandidates & res,
IFileCachePriority::IteratorPtr reservee,
FinalizeEvictionFunc & finalize_eviction_func,
const UserID & user_id,
const CacheGuard::Lock &) override;
void shuffle(const CacheGuard::Lock &) override;
std::vector<FileSegmentInfo> dump(const CacheGuard::Lock &) override;
struct LRUPriorityDump : public IPriorityDump
{
std::vector<FileSegmentInfo> infos;
explicit LRUPriorityDump(const std::vector<FileSegmentInfo> & infos_) : infos(infos_) {}
void merge(const LRUPriorityDump & other) { infos.insert(infos.end(), other.infos.begin(), other.infos.end()); }
};
PriorityDumpPtr dump(const CacheGuard::Lock &) override;
void pop(const CacheGuard::Lock & lock) { remove(queue.begin(), lock); }
bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CacheGuard::Lock &) override;
private:
void updateElementsCount(int64_t num);
void updateSize(int64_t size);
class LRUIterator;
using LRUQueue = std::list<EntryPtr>;
friend class SLRUFileCachePriority;
LRUQueue queue;
Poco::Logger * log = &Poco::Logger::get("LRUFileCachePriority");
StatePtr state;
std::atomic<size_t> current_size = 0;
/// current_elements_num is not always equal to queue.size()
/// because of invalidated entries.
std::atomic<size_t> current_elements_num = 0;
void updateElementsCount(int64_t num);
void updateSize(int64_t size);
bool canFit(size_t size, size_t released_size_assumption, size_t released_elements_assumption, const CacheGuard::Lock &) const;

View File

@ -26,6 +26,7 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
extern const int FILECACHE_ACCESS_DENIED;
}
FileSegmentMetadata::FileSegmentMetadata(FileSegmentPtr && file_segment_)
@ -58,22 +59,36 @@ size_t FileSegmentMetadata::size() const
KeyMetadata::KeyMetadata(
const Key & key_,
const std::string & key_path_,
CleanupQueuePtr cleanup_queue_,
DownloadQueuePtr download_queue_,
Poco::Logger * log_,
std::shared_mutex & key_prefix_directory_mutex_,
const UserInfo & user_,
const CacheMetadata * cache_metadata_,
bool created_base_directory_)
: key(key_)
, key_path(key_path_)
, cleanup_queue(cleanup_queue_)
, download_queue(download_queue_)
, key_prefix_directory_mutex(key_prefix_directory_mutex_)
, user(user_)
, cache_metadata(cache_metadata_)
, created_base_directory(created_base_directory_)
, log(log_)
{
if (created_base_directory)
chassert(fs::exists(key_path));
if (user_ == FileCache::getInternalUser())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create key metadata with internal user id");
if (!user_.weight.has_value())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create key metadata without user weight");
chassert(!created_base_directory || fs::exists(getPath()));
}
bool KeyMetadata::checkAccess(const UserID & user_id_) const
{
return user_id_ == user.user_id || user_id_ == FileCache::getInternalUser().user_id;
}
void KeyMetadata::assertAccess(const UserID & user_id_) const
{
if (!checkAccess(user_id_))
{
throw Exception(ErrorCodes::FILECACHE_ACCESS_DENIED,
"Metadata for key {} belongs to user {}, but user {} requested it",
key.toString(), user.user_id, user_id_);
}
}
LockedKeyPtr KeyMetadata::lock()
@ -108,8 +123,8 @@ bool KeyMetadata::createBaseDirectory()
{
try
{
std::shared_lock lock(key_prefix_directory_mutex);
fs::create_directories(key_path);
std::shared_lock lock(cache_metadata->key_prefix_directory_mutex);
fs::create_directories(getPath());
}
catch (const fs::filesystem_error & e)
{
@ -117,7 +132,7 @@ bool KeyMetadata::createBaseDirectory()
if (e.code() == std::errc::no_space_on_device)
{
LOG_TRACE(log, "Failed to create base directory for key {}, "
LOG_TRACE(cache_metadata->log, "Failed to create base directory for key {}, "
"because no space left on device", key);
return false;
@ -128,16 +143,30 @@ bool KeyMetadata::createBaseDirectory()
return true;
}
std::string KeyMetadata::getFileSegmentPath(const FileSegment & file_segment) const
std::string KeyMetadata::getPath() const
{
return fs::path(key_path)
/ CacheMetadata::getFileNameForFileSegment(file_segment.offset(), file_segment.getKind());
return cache_metadata->getKeyPath(key, user);
}
CacheMetadata::CacheMetadata(const std::string & path_, size_t background_download_queue_size_limit_, size_t background_download_threads_)
std::string KeyMetadata::getFileSegmentPath(const FileSegment & file_segment) const
{
return cache_metadata->getFileSegmentPath(key, file_segment.offset(), file_segment.getKind(), user);
}
Poco::Logger * KeyMetadata::logger() const
{
return cache_metadata->log;
}
CacheMetadata::CacheMetadata(
const std::string & path_,
size_t background_download_queue_size_limit_,
size_t background_download_threads_,
bool write_cache_per_user_directory_)
: path(path_)
, cleanup_queue(std::make_shared<CleanupQueue>())
, download_queue(std::make_shared<DownloadQueue>(background_download_queue_size_limit_))
, write_cache_per_user_directory(write_cache_per_user_directory_)
, log(&Poco::Logger::get("CacheMetadata"))
, download_threads_num(background_download_threads_)
{
@ -157,15 +186,26 @@ String CacheMetadata::getFileNameForFileSegment(size_t offset, FileSegmentKind s
return std::to_string(offset) + file_suffix;
}
String CacheMetadata::getPathForFileSegment(const Key & key, size_t offset, FileSegmentKind segment_kind) const
String CacheMetadata::getFileSegmentPath(
const Key & key,
size_t offset,
FileSegmentKind segment_kind,
const UserInfo & user) const
{
return fs::path(getPathForKey(key)) / getFileNameForFileSegment(offset, segment_kind);
return fs::path(getKeyPath(key, user)) / getFileNameForFileSegment(offset, segment_kind);
}
String CacheMetadata::getPathForKey(const Key & key) const
String CacheMetadata::getKeyPath(const Key & key, const UserInfo & user) const
{
const auto key_str = key.toString();
return fs::path(path) / key_str.substr(0, 3) / key_str;
if (write_cache_per_user_directory)
{
return fs::path(path) / fmt::format("{}.{}", user.user_id, user.weight.value()) / key.toString();
}
else
{
const auto key_str = key.toString();
return fs::path(path) / key_str.substr(0, 3) / key_str;
}
}
CacheMetadataGuard::Lock CacheMetadata::MetadataBucket::lock() const
@ -183,9 +223,10 @@ CacheMetadata::MetadataBucket & CacheMetadata::getMetadataBucket(const Key & key
LockedKeyPtr CacheMetadata::lockKeyMetadata(
const FileCacheKey & key,
KeyNotFoundPolicy key_not_found_policy,
const UserInfo & user,
bool is_initial_load)
{
auto key_metadata = getKeyMetadata(key, key_not_found_policy, is_initial_load);
auto key_metadata = getKeyMetadata(key, key_not_found_policy, user, is_initial_load);
if (!key_metadata)
return nullptr;
@ -218,12 +259,13 @@ LockedKeyPtr CacheMetadata::lockKeyMetadata(
/// Now we are at the case when the key was removed (key_state == KeyMetadata::KeyState::REMOVED)
/// but we need to return empty key (key_not_found_policy == KeyNotFoundPolicy::CREATE_EMPTY)
/// Retry
return lockKeyMetadata(key, key_not_found_policy);
return lockKeyMetadata(key, key_not_found_policy, user);
}
KeyMetadataPtr CacheMetadata::getKeyMetadata(
const Key & key,
KeyNotFoundPolicy key_not_found_policy,
const UserInfo & user,
bool is_initial_load)
{
auto & bucket = getMetadataBucket(key);
@ -240,10 +282,10 @@ KeyMetadataPtr CacheMetadata::getKeyMetadata(
return nullptr;
it = bucket.emplace(
key, std::make_shared<KeyMetadata>(
key, getPathForKey(key), cleanup_queue, download_queue, log, key_prefix_directory_mutex, is_initial_load)).first;
key, std::make_shared<KeyMetadata>(key, user, this, is_initial_load)).first;
}
it->second->assertAccess(user.user_id);
return it->second;
}
@ -255,13 +297,16 @@ bool CacheMetadata::isEmpty() const
return true;
}
void CacheMetadata::iterate(IterateFunc && func)
void CacheMetadata::iterate(IterateFunc && func, const KeyMetadata::UserID & user_id)
{
for (auto & bucket : metadata_buckets)
{
auto lk = bucket.lock();
for (auto & [key, key_metadata] : bucket)
{
if (!key_metadata->checkAccess(user_id))
continue;
auto locked_key = key_metadata->lockNoStateCheck();
const auto key_state = locked_key->getKeyState();
@ -279,13 +324,16 @@ void CacheMetadata::iterate(IterateFunc && func)
}
}
void CacheMetadata::removeAllKeys(bool if_releasable)
void CacheMetadata::removeAllKeys(bool if_releasable, const UserID & user_id)
{
for (auto & bucket : metadata_buckets)
{
auto lock = bucket.lock();
for (auto it = bucket.begin(); it != bucket.end();)
{
if (!it->second->checkAccess(user_id))
continue;
auto locked_key = it->second->lockNoStateCheck();
if (locked_key->getKeyState() == KeyMetadata::KeyState::ACTIVE)
{
@ -301,7 +349,7 @@ void CacheMetadata::removeAllKeys(bool if_releasable)
}
}
void CacheMetadata::removeKey(const Key & key, bool if_exists, bool if_releasable)
void CacheMetadata::removeKey(const Key & key, bool if_exists, bool if_releasable, const UserID & user_id)
{
auto & bucket = getMetadataBucket(key);
auto lock = bucket.lock();
@ -314,6 +362,7 @@ void CacheMetadata::removeKey(const Key & key, bool if_exists, bool if_releasabl
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key: {}", key);
}
it->second->assertAccess(user_id);
auto locked_key = it->second->lockNoStateCheck();
auto state = locked_key->getKeyState();
if (state != KeyMetadata::KeyState::ACTIVE)
@ -321,7 +370,8 @@ void CacheMetadata::removeKey(const Key & key, bool if_exists, bool if_releasabl
if (if_exists)
return;
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key: {} (state: {})", key, magic_enum::enum_name(state));
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"No such key: {} (state: {})", key, magic_enum::enum_name(state));
}
bool removed_all = locked_key->removeAllFileSegments(if_releasable);
@ -346,7 +396,7 @@ CacheMetadata::removeEmptyKey(
LOG_DEBUG(log, "Key {} is removed from metadata", key);
const fs::path key_directory = getPathForKey(key);
const fs::path key_directory = getKeyPath(key, locked_key.getKeyMetadata()->user);
const fs::path key_prefix_directory = key_directory.parent_path();
try
@ -383,7 +433,7 @@ CacheMetadata::removeEmptyKey(
class CleanupQueue
{
friend struct CacheMetadata;
friend class CacheMetadata;
public:
void add(const FileCacheKey & key)
{
@ -467,7 +517,7 @@ void CacheMetadata::cleanupThreadFunc()
class DownloadQueue
{
friend struct CacheMetadata;
friend class CacheMetadata;
public:
explicit DownloadQueue(size_t queue_size_limit_) : queue_size_limit(queue_size_limit_) {}
@ -504,7 +554,10 @@ private:
struct DownloadInfo
{
DownloadInfo(const CacheMetadata::Key & key_, const size_t & offset_, const std::weak_ptr<FileSegment> & file_segment_)
DownloadInfo(
const CacheMetadata::Key & key_,
const size_t & offset_,
const std::weak_ptr<FileSegment> & file_segment_)
: key(key_), offset(offset_), file_segment(file_segment_) {}
CacheMetadata::Key key;
@ -556,7 +609,7 @@ void CacheMetadata::downloadThreadFunc(const bool & stop_flag)
try
{
{
auto locked_key = lockKeyMetadata(key, KeyNotFoundPolicy::RETURN_NULL);
auto locked_key = lockKeyMetadata(key, KeyNotFoundPolicy::RETURN_NULL, FileCache::getInternalUser());
if (!locked_key)
continue;
@ -616,7 +669,7 @@ bool CacheMetadata::setBackgroundDownloadQueueSizeLimit(size_t size)
return download_queue->setQueueLimit(size);
}
void CacheMetadata::downloadImpl(FileSegment & file_segment, std::optional<Memory<>> & memory)
void CacheMetadata::downloadImpl(FileSegment & file_segment, std::optional<Memory<>> & memory) const
{
LOG_TEST(
log, "Downloading {} bytes for file segment {}",
@ -685,7 +738,8 @@ void CacheMetadata::startup()
for (size_t i = 0; i < download_threads_num; ++i)
{
download_threads.emplace_back(std::make_shared<DownloadThread>());
download_threads.back()->thread = std::make_unique<ThreadFromGlobalPool>([this, thread = download_threads.back()] { downloadThreadFunc(thread->stop_flag); });
download_threads.back()->thread = std::make_unique<ThreadFromGlobalPool>(
[this, thread = download_threads.back()] { downloadThreadFunc(thread->stop_flag); });
}
cleanup_thread = std::make_unique<ThreadFromGlobalPool>([this]{ cleanupThreadFunc(); });
}
@ -760,6 +814,16 @@ bool CacheMetadata::setBackgroundDownloadThreads(size_t threads_num)
return true;
}
bool KeyMetadata::addToDownloadQueue(FileSegmentPtr file_segment)
{
return cache_metadata->download_queue->add(file_segment);
}
void KeyMetadata::addToCleanupQueue()
{
cache_metadata->cleanup_queue->add(key);
}
LockedKey::LockedKey(std::shared_ptr<KeyMetadata> key_metadata_)
: key_metadata(key_metadata_)
, lock(key_metadata->guard.lock())
@ -778,8 +842,8 @@ LockedKey::~LockedKey()
/// See comment near cleanupThreadFunc() for more details.
key_metadata->key_state = KeyMetadata::KeyState::REMOVING;
LOG_DEBUG(key_metadata->log, "Submitting key {} for removal", getKey());
key_metadata->cleanup_queue->add(getKey());
LOG_DEBUG(key_metadata->logger(), "Submitting key {} for removal", getKey());
key_metadata->addToCleanupQueue();
}
void LockedKey::removeFromCleanupQueue()
@ -841,7 +905,10 @@ KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, bool can_be_br
return removeFileSegmentImpl(it, file_segment->lock(), can_be_broken);
}
KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegmentGuard::Lock & segment_lock, bool can_be_broken)
KeyMetadata::iterator LockedKey::removeFileSegment(
size_t offset,
const FileSegmentGuard::Lock & segment_lock,
bool can_be_broken)
{
auto it = key_metadata->find(offset);
if (it == key_metadata->end())
@ -850,12 +917,15 @@ KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegm
return removeFileSegmentImpl(it, segment_lock, can_be_broken);
}
KeyMetadata::iterator LockedKey::removeFileSegmentImpl(KeyMetadata::iterator it, const FileSegmentGuard::Lock & segment_lock, bool can_be_broken)
KeyMetadata::iterator LockedKey::removeFileSegmentImpl(
KeyMetadata::iterator it,
const FileSegmentGuard::Lock & segment_lock,
bool can_be_broken)
{
auto file_segment = it->second->file_segment;
LOG_DEBUG(
key_metadata->log, "Remove from cache. Key: {}, offset: {}, size: {}",
key_metadata->logger(), "Remove from cache. Key: {}, offset: {}, size: {}",
getKey(), file_segment->offset(), file_segment->reserved_size);
chassert(can_be_broken || file_segment->assertCorrectnessUnlocked(segment_lock));
@ -880,14 +950,14 @@ KeyMetadata::iterator LockedKey::removeFileSegmentImpl(KeyMetadata::iterator it,
OpenedFileCache::instance().remove(path, flags);
OpenedFileCache::instance().remove(path, flags | O_DIRECT);
LOG_TEST(key_metadata->log, "Removed file segment at path: {}", path);
LOG_TEST(key_metadata->logger(), "Removed file segment at path: {}", path);
}
else if (file_segment->downloaded_size && !can_be_broken)
{
#ifdef ABORT_ON_LOGICAL_ERROR
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected path {} to exist", path);
#else
LOG_WARNING(key_metadata->log, "Expected path {} to exist, while removing {}:{}",
LOG_WARNING(key_metadata->logger(), "Expected path {} to exist, while removing {}:{}",
path, getKey(), file_segment->offset());
#endif
}
@ -942,7 +1012,7 @@ bool LockedKey::addToDownloadQueue(size_t offset, const FileSegmentGuard::Lock &
auto it = key_metadata->find(offset);
if (it == key_metadata->end())
throw Exception(ErrorCodes::LOGICAL_ERROR, "There is not offset {}", offset);
return key_metadata->download_queue->add(it->second->file_segment);
return key_metadata->addToDownloadQueue(it->second->file_segment);
}
std::optional<FileSegment::Range> LockedKey::hasIntersectingRange(const FileSegment::Range & range) const
@ -1042,7 +1112,7 @@ std::vector<FileSegment::Info> LockedKey::sync()
if (!fs::exists(path))
{
LOG_WARNING(
key_metadata->log,
key_metadata->logger(),
"File segment has DOWNLOADED state, but file does not exist ({})",
file_segment->getInfoForLog());
@ -1061,7 +1131,7 @@ std::vector<FileSegment::Info> LockedKey::sync()
}
LOG_WARNING(
key_metadata->log,
key_metadata->logger(),
"File segment has unexpected size. Having {}, expected {} ({})",
actual_size, expected_size, file_segment->getInfoForLog());

View File

@ -15,7 +15,9 @@ class CleanupQueue;
using CleanupQueuePtr = std::shared_ptr<CleanupQueue>;
class DownloadQueue;
using DownloadQueuePtr = std::shared_ptr<DownloadQueue>;
using FileSegmentsHolderPtr = std::unique_ptr<FileSegmentsHolder>;
class CacheMetadata;
struct FileSegmentMetadata : private boost::noncopyable
@ -43,17 +45,18 @@ struct KeyMetadata : private std::map<size_t, FileSegmentMetadataPtr>,
private boost::noncopyable,
public std::enable_shared_from_this<KeyMetadata>
{
friend class CacheMetadata;
friend struct LockedKey;
using Key = FileCacheKey;
using iterator = iterator;
using UserInfo = FileCacheUserInfo;
using UserID = UserInfo::UserID;
KeyMetadata(
const Key & key_,
const std::string & key_path_,
CleanupQueuePtr cleanup_queue_,
DownloadQueuePtr download_queue_,
Poco::Logger * log_,
std::shared_mutex & key_prefix_directory_mutex_,
const UserInfo & user_id_,
const CacheMetadata * cache_metadata_,
bool created_base_directory_ = false);
enum class KeyState
@ -64,19 +67,22 @@ struct KeyMetadata : private std::map<size_t, FileSegmentMetadataPtr>,
};
const Key key;
const std::string key_path;
const UserInfo user;
LockedKeyPtr lock();
/// Return nullptr if key has non-ACTIVE state.
LockedKeyPtr tryLock();
LockedKeyPtr lockNoStateCheck();
bool createBaseDirectory();
std::string getPath() const;
std::string getFileSegmentPath(const FileSegment & file_segment) const;
bool checkAccess(const UserID & user_id_) const;
void assertAccess(const UserID & user_id_) const;
/// This method is used for loadMetadata() on server startup,
/// where we know there is no concurrency on Key and we do not want therefore taking a KeyGuard::Lock,
/// therefore we use this Unlocked version. This method should not be used anywhere else.
@ -85,41 +91,52 @@ struct KeyMetadata : private std::map<size_t, FileSegmentMetadataPtr>,
size_t sizeUnlocked() const { return size(); }
private:
const CacheMetadata * cache_metadata;
KeyState key_state = KeyState::ACTIVE;
KeyGuard guard;
const CleanupQueuePtr cleanup_queue;
const DownloadQueuePtr download_queue;
std::shared_mutex & key_prefix_directory_mutex;
std::atomic<bool> created_base_directory = false;
Poco::Logger * log;
LockedKeyPtr lockNoStateCheck();
Poco::Logger * logger() const;
bool addToDownloadQueue(FileSegmentPtr file_segment);
void addToCleanupQueue();
};
using KeyMetadataPtr = std::shared_ptr<KeyMetadata>;
struct CacheMetadata
class CacheMetadata : private boost::noncopyable
{
friend struct KeyMetadata;
public:
using Key = FileCacheKey;
using IterateFunc = std::function<void(LockedKey &)>;
using UserInfo = FileCacheUserInfo;
using UserID = UserInfo::UserID;
explicit CacheMetadata(const std::string & path_, size_t background_download_queue_size_limit_, size_t background_download_threads_);
explicit CacheMetadata(
const std::string & path_,
size_t background_download_queue_size_limit_,
size_t background_download_threads_,
bool write_cache_per_user_directory_);
void startup();
bool isEmpty() const;
const String & getBaseDirectory() const { return path; }
String getPathForFileSegment(
String getKeyPath(const Key & key, const UserInfo & user) const;
String getFileSegmentPath(
const Key & key,
size_t offset,
FileSegmentKind segment_kind) const;
FileSegmentKind segment_kind,
const UserInfo & user) const;
String getPathForKey(const Key & key) const;
static String getFileNameForFileSegment(size_t offset, FileSegmentKind segment_kind);
void iterate(IterateFunc && func);
bool isEmpty() const;
void iterate(IterateFunc && func, const UserID & user_id);
enum class KeyNotFoundPolicy
{
@ -132,15 +149,17 @@ public:
KeyMetadataPtr getKeyMetadata(
const Key & key,
KeyNotFoundPolicy key_not_found_policy,
const UserInfo & user,
bool is_initial_load = false);
LockedKeyPtr lockKeyMetadata(
const Key & key,
KeyNotFoundPolicy key_not_found_policy,
const UserInfo & user,
bool is_initial_load = false);
void removeKey(const Key & key, bool if_exists, bool if_releasable);
void removeAllKeys(bool if_releasable);
void removeKey(const Key & key, bool if_exists, bool if_releasable, const UserID & user_id);
void removeAllKeys(bool if_releasable, const UserID & user_id);
void shutdown();
@ -151,12 +170,15 @@ public:
bool isBackgroundDownloadEnabled();
private:
const std::string path; /// Cache base path
static constexpr size_t buckets_num = 1024;
const std::string path;
const CleanupQueuePtr cleanup_queue;
const DownloadQueuePtr download_queue;
const bool write_cache_per_user_directory;
std::shared_mutex key_prefix_directory_mutex;
Poco::Logger * log;
mutable std::shared_mutex key_prefix_directory_mutex;
struct MetadataBucket : public std::unordered_map<FileCacheKey, KeyMetadataPtr>
{
@ -165,7 +187,6 @@ private:
mutable CacheMetadataGuard guard;
};
static constexpr size_t buckets_num = 1024;
std::vector<MetadataBucket> metadata_buckets{buckets_num};
struct DownloadThread
@ -173,13 +194,15 @@ private:
std::unique_ptr<ThreadFromGlobalPool> thread;
bool stop_flag{false};
};
std::vector<std::shared_ptr<DownloadThread>> download_threads;
std::atomic<size_t> download_threads_num;
std::atomic<size_t> download_threads_num;
std::vector<std::shared_ptr<DownloadThread>> download_threads;
std::unique_ptr<ThreadFromGlobalPool> cleanup_thread;
static String getFileNameForFileSegment(size_t offset, FileSegmentKind segment_kind);
MetadataBucket & getMetadataBucket(const Key & key);
void downloadImpl(FileSegment & file_segment, std::optional<Memory<>> & memory);
void downloadImpl(FileSegment & file_segment, std::optional<Memory<>> & memory) const;
MetadataBucket::iterator removeEmptyKey(
MetadataBucket & bucket,
MetadataBucket::iterator it,

View File

@ -1,5 +1,6 @@
#include <Interpreters/Cache/QueryLimit.h>
#include <Interpreters/Cache/Metadata.h>
#include <Interpreters/Cache/FileCache.h>
namespace DB
{
@ -68,9 +69,10 @@ void FileCacheQueryLimit::QueryContext::add(
KeyMetadataPtr key_metadata,
size_t offset,
size_t size,
const FileCache::UserInfo & user,
const CacheGuard::Lock & lock)
{
auto it = getPriority().add(key_metadata, offset, size, lock);
auto it = getPriority().add(key_metadata, offset, size, user, lock);
auto [_, inserted] = records.emplace(FileCacheKeyAndOffset{key_metadata->key, offset}, it);
if (!inserted)
{
@ -107,4 +109,27 @@ IFileCachePriority::IteratorPtr FileCacheQueryLimit::QueryContext::tryGet(
}
FileCacheQueryLimit::QueryContextHolder::QueryContextHolder(
const String & query_id_,
FileCache * cache_,
FileCacheQueryLimit * query_limit_,
FileCacheQueryLimit::QueryContextPtr context_)
: query_id(query_id_)
, cache(cache_)
, query_limit(query_limit_)
, context(context_)
{
}
FileCacheQueryLimit::QueryContextHolder::~QueryContextHolder()
{
/// If only the query_map and the current holder hold the context_query,
/// the query has been completed and the query_context is released.
if (context && context.use_count() == 2)
{
auto lock = cache->lockCache();
query_limit->removeQueryContext(query_id, lock);
}
}
}

View File

@ -44,6 +44,7 @@ public:
KeyMetadataPtr key_metadata,
size_t offset,
size_t size,
const FileCacheUserInfo & user,
const CacheGuard::Lock &);
void remove(
@ -58,6 +59,21 @@ public:
const bool recache_on_query_limit_exceeded;
};
struct QueryContextHolder : private boost::noncopyable
{
QueryContextHolder(const String & query_id_, FileCache * cache_, FileCacheQueryLimit * query_limit_, QueryContextPtr context_);
QueryContextHolder() = default;
~QueryContextHolder();
String query_id;
FileCache * cache;
FileCacheQueryLimit * query_limit;
QueryContextPtr context;
};
using QueryContextHolderPtr = std::unique_ptr<QueryContextHolder>;
private:
using QueryContextMap = std::unordered_map<String, QueryContextPtr>;
QueryContextMap query_map;

View File

@ -21,11 +21,13 @@ namespace
SLRUFileCachePriority::SLRUFileCachePriority(
size_t max_size_,
size_t max_elements_,
double size_ratio_)
double size_ratio_,
LRUFileCachePriority::StatePtr probationary_state_,
LRUFileCachePriority::StatePtr protected_state_)
: IFileCachePriority(max_size_, max_elements_)
, size_ratio(size_ratio_)
, protected_queue(LRUFileCachePriority(getRatio(max_size_, size_ratio), getRatio(max_elements_, size_ratio)))
, probationary_queue(LRUFileCachePriority(getRatio(max_size_, 1 - size_ratio), getRatio(max_elements_, 1 - size_ratio)))
, protected_queue(LRUFileCachePriority(getRatio(max_size_, size_ratio), getRatio(max_elements_, size_ratio), protected_state_))
, probationary_queue(LRUFileCachePriority(getRatio(max_size_, 1 - size_ratio), getRatio(max_elements_, 1 - size_ratio), probationary_state_))
{
LOG_DEBUG(
log, "Using probationary queue size: {}, protected queue size: {}",
@ -42,15 +44,32 @@ size_t SLRUFileCachePriority::getElementsCount(const CacheGuard::Lock & lock) co
return protected_queue.getElementsCount(lock) + probationary_queue.getElementsCount(lock);
}
bool SLRUFileCachePriority::canFit(size_t size, const CacheGuard::Lock & lock) const
bool SLRUFileCachePriority::canFit( /// NOLINT
size_t size,
const CacheGuard::Lock & lock,
IteratorPtr reservee,
bool best_effort) const
{
return probationary_queue.canFit(size, lock) || protected_queue.canFit(size, lock);
if (best_effort)
return probationary_queue.canFit(size, lock) || protected_queue.canFit(size, lock);
if (reservee)
{
const auto * slru_iterator = assert_cast<SLRUIterator *>(reservee.get());
if (slru_iterator->is_protected)
return protected_queue.canFit(size, lock);
else
return probationary_queue.canFit(size, lock);
}
else
return probationary_queue.canFit(size, lock);
}
IFileCachePriority::IteratorPtr SLRUFileCachePriority::add( /// NOLINT
KeyMetadataPtr key_metadata,
size_t offset,
size_t size,
const UserInfo &,
const CacheGuard::Lock & lock,
bool is_startup)
{
@ -83,13 +102,14 @@ bool SLRUFileCachePriority::collectCandidatesForEviction(
EvictionCandidates & res,
IFileCachePriority::IteratorPtr reservee,
FinalizeEvictionFunc & finalize_eviction_func,
const UserID & user_id,
const CacheGuard::Lock & lock)
{
/// If `it` is nullptr, then it is the first space reservation attempt
/// for a corresponding file segment, so it will be directly put into probationary queue.
if (!reservee)
{
return probationary_queue.collectCandidatesForEviction(size, stat, res, reservee, finalize_eviction_func, lock);
return probationary_queue.collectCandidatesForEviction(size, stat, res, reservee, finalize_eviction_func, user_id, lock);
}
/// If `it` not nullptr (e.g. is already in some queue),
@ -97,7 +117,7 @@ bool SLRUFileCachePriority::collectCandidatesForEviction(
/// (in order to know where we need to free space).
if (!assert_cast<SLRUIterator *>(reservee.get())->is_protected)
{
return probationary_queue.collectCandidatesForEviction(size, stat, res, reservee, finalize_eviction_func, lock);
return probationary_queue.collectCandidatesForEviction(size, stat, res, reservee, finalize_eviction_func, user_id, lock);
}
/// Entry is in protected queue.
@ -114,13 +134,13 @@ bool SLRUFileCachePriority::collectCandidatesForEviction(
FileCacheReserveStat downgrade_stat;
FinalizeEvictionFunc noop;
if (!protected_queue.collectCandidatesForEviction(size, downgrade_stat, *downgrade_candidates, reservee, noop, lock))
if (!protected_queue.collectCandidatesForEviction(size, downgrade_stat, *downgrade_candidates, reservee, noop, user_id, lock))
return false;
const size_t size_to_downgrade = downgrade_stat.stat.releasable_size;
if (!probationary_queue.canFit(size_to_downgrade, lock)
&& !probationary_queue.collectCandidatesForEviction(size_to_downgrade, stat, res, reservee, noop, lock))
&& !probationary_queue.collectCandidatesForEviction(size_to_downgrade, stat, res, reservee, noop, user_id, lock))
return false;
finalize_eviction_func = [=, this](const CacheGuard::Lock & lk) mutable
@ -168,7 +188,7 @@ void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const Cach
FileCacheReserveStat downgrade_stat;
FinalizeEvictionFunc noop;
if (!protected_queue.collectCandidatesForEviction(size, downgrade_stat, downgrade_candidates, {}, noop, lock))
if (!protected_queue.collectCandidatesForEviction(size, downgrade_stat, downgrade_candidates, {}, noop, "", lock))
{
/// We cannot make space for entry to be moved to protected queue
/// (not enough releasable file segments).
@ -191,7 +211,7 @@ void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const Cach
if (size_to_free)
{
if (!probationary_queue.collectCandidatesForEviction(size_to_free, stat, eviction_candidates, {}, noop, lock))
if (!probationary_queue.collectCandidatesForEviction(size_to_free, stat, eviction_candidates, {}, noop, {}, lock))
{
/// "downgrade" candidates cannot be moved to probationary queue,
/// so entry cannot be moved to protected queue as well.
@ -222,11 +242,11 @@ void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const Cach
iterator.is_protected = true;
}
std::vector<FileSegmentInfo> SLRUFileCachePriority::dump(const CacheGuard::Lock & lock)
IFileCachePriority::PriorityDumpPtr SLRUFileCachePriority::dump(const CacheGuard::Lock & lock)
{
auto res = probationary_queue.dump(lock);
auto part_res = protected_queue.dump(lock);
res.insert(res.end(), part_res.begin(), part_res.end());
auto res = dynamic_pointer_cast<LRUFileCachePriority::LRUPriorityDump>(probationary_queue.dump(lock));
auto part_res = dynamic_pointer_cast<LRUFileCachePriority::LRUPriorityDump>(protected_queue.dump(lock));
res->merge(*part_res);
return res;
}

View File

@ -14,18 +14,28 @@ class SLRUFileCachePriority : public IFileCachePriority
public:
class SLRUIterator;
SLRUFileCachePriority(size_t max_size_, size_t max_elements_, double size_ratio_);
SLRUFileCachePriority(
size_t max_size_,
size_t max_elements_,
double size_ratio_,
LRUFileCachePriority::StatePtr probationary_state_ = nullptr,
LRUFileCachePriority::StatePtr protected_state_ = nullptr);
size_t getSize(const CacheGuard::Lock & lock) const override;
size_t getElementsCount(const CacheGuard::Lock &) const override;
bool canFit(size_t size, const CacheGuard::Lock &) const override;
bool canFit( /// NOLINT
size_t size,
const CacheGuard::Lock &,
IteratorPtr reservee = nullptr,
bool best_effort = false) const override;
IteratorPtr add( /// NOLINT
KeyMetadataPtr key_metadata,
size_t offset,
size_t size,
const UserInfo & user,
const CacheGuard::Lock &,
bool is_startup = false) override;
@ -35,11 +45,12 @@ public:
EvictionCandidates & res,
IFileCachePriority::IteratorPtr reservee,
FinalizeEvictionFunc & finalize_eviction_func,
const UserID & user_id,
const CacheGuard::Lock &) override;
void shuffle(const CacheGuard::Lock &) override;
std::vector<FileSegmentInfo> dump(const CacheGuard::Lock &) override;
PriorityDumpPtr dump(const CacheGuard::Lock &) override;
bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CacheGuard::Lock &) override;
@ -79,7 +90,10 @@ private:
SLRUFileCachePriority * cache_priority;
LRUFileCachePriority::LRUIterator lru_iterator;
const EntryPtr entry;
bool is_protected;
/// Atomic,
/// but needed only in order to do FileSegment::getInfo() without any lock,
/// which is done for system tables and logging.
std::atomic<bool> is_protected;
};
}

View File

@ -0,0 +1,24 @@
#pragma once
#include <Core/UUID.h>
namespace DB
{
struct FileCacheUserInfo
{
using UserID = std::string;
using Weight = UInt64;
UserID user_id;
std::optional<Weight> weight = std::nullopt;
FileCacheUserInfo() = default;
explicit FileCacheUserInfo(const UserID & user_id_) : user_id(user_id_) {}
FileCacheUserInfo(const UserID & user_id_, const Weight & weight_) : user_id(user_id_), weight(weight_) {}
bool operator ==(const FileCacheUserInfo & other) const { return user_id == other.user_id; }
};
}

View File

@ -19,7 +19,7 @@ namespace ErrorCodes
}
WriteBufferToFileSegment::WriteBufferToFileSegment(FileSegment * file_segment_)
: WriteBufferFromFileDecorator(std::make_unique<WriteBufferFromFile>(file_segment_->getPathInLocalCache()))
: WriteBufferFromFileDecorator(std::make_unique<WriteBufferFromFile>(file_segment_->getPath()))
, file_segment(file_segment_)
{
}
@ -27,7 +27,7 @@ WriteBufferToFileSegment::WriteBufferToFileSegment(FileSegment * file_segment_)
WriteBufferToFileSegment::WriteBufferToFileSegment(FileSegmentsHolderPtr segment_holder_)
: WriteBufferFromFileDecorator(
segment_holder_->size() == 1
? std::make_unique<WriteBufferFromFile>(segment_holder_->front().getPathInLocalCache())
? std::make_unique<WriteBufferFromFile>(segment_holder_->front().getPath())
: throw Exception(ErrorCodes::LOGICAL_ERROR, "WriteBufferToFileSegment can be created only from single segment"))
, file_segment(&segment_holder_->front())
, segment_holder(std::move(segment_holder_))
@ -84,7 +84,7 @@ void WriteBufferToFileSegment::nextImpl()
std::unique_ptr<ReadBuffer> WriteBufferToFileSegment::getReadBufferImpl()
{
finalize();
return std::make_unique<ReadBufferFromFile>(file_segment->getPathInLocalCache());
return std::make_unique<ReadBufferFromFile>(file_segment->getPath());
}
WriteBufferToFileSegment::~WriteBufferToFileSegment()

View File

@ -117,13 +117,13 @@ void SelectStreamFactory::createForShard(
std::vector<QueryPlanPtr> & local_plans,
Shards & remote_shards,
UInt32 shard_count,
bool parallel_replicas_enabled,
AdditionalShardFilterGenerator shard_filter_generator)
bool parallel_replicas_enabled)
{
auto it = objects_by_shard.find(shard_info.shard_num);
if (it != objects_by_shard.end())
replaceMissedSubcolumnsByConstants(storage_snapshot->object_columns, it->second, query_ast);
auto emplace_local_stream = [&]()
{
local_plans.emplace_back(createLocalPlan(
@ -139,7 +139,6 @@ void SelectStreamFactory::createForShard(
.shard_info = shard_info,
.lazy = lazy,
.local_delay = local_delay,
.shard_filter_generator = std::move(shard_filter_generator),
});
};

View File

@ -40,7 +40,6 @@ ASTPtr rewriteSelectQuery(
ASTPtr table_function_ptr = nullptr);
using ColumnsDescriptionByShardNum = std::unordered_map<UInt32, ColumnsDescription>;
using AdditionalShardFilterGenerator = std::function<ASTPtr(uint64_t)>;
class SelectStreamFactory
{
@ -60,7 +59,6 @@ public:
/// (When there is a local replica with big delay).
bool lazy = false;
time_t local_delay = 0;
AdditionalShardFilterGenerator shard_filter_generator{};
};
using Shards = std::vector<Shard>;
@ -80,8 +78,7 @@ public:
std::vector<QueryPlanPtr> & local_plans,
Shards & remote_shards,
UInt32 shard_count,
bool parallel_replicas_enabled,
AdditionalShardFilterGenerator shard_filter_generator);
bool parallel_replicas_enabled);
const Block header;
const ColumnsDescriptionByShardNum objects_by_shard;

View File

@ -158,13 +158,6 @@ ContextMutablePtr updateSettingsForCluster(const Cluster & cluster,
new_settings.timeout_overflow_mode = settings.timeout_overflow_mode_leaf;
}
/// in case of parallel replicas custom key use round robing load balancing
/// so custom key partitions will be spread over nodes in round-robin fashion
if (context->canUseParallelReplicasCustomKey(cluster) && !settings.load_balancing.changed)
{
new_settings.load_balancing = LoadBalancing::ROUND_ROBIN;
}
auto new_context = Context::createCopy(context);
new_context->setSettings(new_settings);
return new_context;
@ -254,6 +247,21 @@ void executeQuery(
visitor.visit(query_ast_for_shard);
}
if (shard_filter_generator)
{
auto shard_filter = shard_filter_generator(shard_info.shard_num);
if (shard_filter)
{
auto & select_query = query_ast_for_shard->as<ASTSelectQuery &>();
auto where_expression = select_query.where();
if (where_expression)
shard_filter = makeASTFunction("and", where_expression, shard_filter);
select_query.setExpression(ASTSelectQuery::Expression::WHERE, std::move(shard_filter));
}
}
// decide for each shard if parallel reading from replicas should be enabled
// according to settings and number of replicas declared per shard
const auto & addresses = cluster->getShardsAddresses().at(i);
@ -268,8 +276,7 @@ void executeQuery(
plans,
remote_shards,
static_cast<UInt32>(shards),
parallel_replicas_enabled,
shard_filter_generator);
parallel_replicas_enabled);
}
if (!remote_shards.empty())
@ -322,7 +329,6 @@ void executeQuery(
void executeQueryWithParallelReplicas(
QueryPlan & query_plan,
const StorageID & main_table,
SelectStreamFactory & stream_factory,
const ASTPtr & query_ast,
ContextPtr context,
@ -414,7 +420,6 @@ void executeQueryWithParallelReplicas(
std::move(coordinator),
stream_factory.header,
stream_factory.processed_stage,
main_table,
new_context,
getThrottler(new_context),
std::move(scalars),

View File

@ -65,12 +65,11 @@ void executeQuery(
const std::string & sharding_key_column_name,
const ClusterPtr & not_optimized_cluster,
const DistributedSettings & distributed_settings,
AdditionalShardFilterGenerator shard_filter_generator);
AdditionalShardFilterGenerator shard_filter_generator = {});
void executeQueryWithParallelReplicas(
QueryPlan & query_plan,
const StorageID & main_table,
SelectStreamFactory & stream_factory,
const ASTPtr & query_ast,
ContextPtr context,

View File

@ -237,6 +237,7 @@ struct ContextSharedPart : boost::noncopyable
String dictionaries_lib_path TSA_GUARDED_BY(mutex); /// Path to the directory with user provided binaries and libraries for external dictionaries.
String user_scripts_path TSA_GUARDED_BY(mutex); /// Path to the directory with user provided scripts.
String filesystem_caches_path TSA_GUARDED_BY(mutex); /// Path to the directory with filesystem caches.
String filesystem_cache_user TSA_GUARDED_BY(mutex);
ConfigurationPtr config TSA_GUARDED_BY(mutex); /// Global configuration settings.
String tmp_path TSA_GUARDED_BY(mutex); /// Path to the temporary files that occur when processing the request.
@ -617,6 +618,7 @@ struct ContextSharedPart : boost::noncopyable
const auto & caches = FileCacheFactory::instance().getAll();
for (const auto & [_, cache] : caches)
cache->cache->deactivateBackgroundOperations();
FileCacheFactory::instance().clear();
{
// Disk selector might not be initialized if there was some error during
@ -888,6 +890,12 @@ String Context::getFilesystemCachesPath() const
return shared->filesystem_caches_path;
}
String Context::getFilesystemCacheUser() const
{
SharedLockGuard lock(shared->mutex);
return shared->filesystem_cache_user;
}
Strings Context::getWarnings() const
{
Strings common_warnings;
@ -995,6 +1003,12 @@ void Context::setFilesystemCachesPath(const String & path)
shared->filesystem_caches_path = path;
}
void Context::setFilesystemCacheUser(const String & user)
{
std::lock_guard lock(shared->mutex);
shared->filesystem_cache_user = user;
}
static void setupTmpPath(Poco::Logger * log, const std::string & path)
try
{
@ -5099,12 +5113,6 @@ bool Context::canUseParallelReplicasOnFollower() const
return canUseTaskBasedParallelReplicas() && getClientInfo().collaborate_with_initiator;
}
bool Context::canUseParallelReplicasCustomKey(const Cluster & cluster) const
{
return settings.max_parallel_replicas > 1 && getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY
&& cluster.getShardCount() == 1 && cluster.getShardsInfo()[0].getAllNodeCount() > 1;
}
void Context::setPreparedSetsCache(const PreparedSetsCachePtr & cache)
{
prepared_sets_cache = cache;

View File

@ -529,6 +529,7 @@ public:
String getDictionariesLibPath() const;
String getUserScriptsPath() const;
String getFilesystemCachesPath() const;
String getFilesystemCacheUser() const;
/// A list of warnings about server configuration to place in `system.warnings` table.
Strings getWarnings() const;
@ -540,6 +541,7 @@ public:
void setTempDataOnDisk(TemporaryDataOnDiskScopePtr temp_data_on_disk_);
void setFilesystemCachesPath(const String & path);
void setFilesystemCacheUser(const String & user);
void setPath(const String & path);
void setFlagsPath(const String & path);
@ -1244,7 +1246,6 @@ public:
bool canUseTaskBasedParallelReplicas() const;
bool canUseParallelReplicasOnInitiator() const;
bool canUseParallelReplicasOnFollower() const;
bool canUseParallelReplicasCustomKey(const Cluster & cluster) const;
enum class ParallelReplicasMode : uint8_t
{

View File

@ -589,8 +589,9 @@ InterpreterSelectQuery::InterpreterSelectQuery(
}
}
else if (auto * distributed = dynamic_cast<StorageDistributed *>(storage.get());
distributed && context->canUseParallelReplicasCustomKey(*distributed->getCluster()))
distributed && canUseCustomKey(settings, *distributed->getCluster(), *context))
{
query_info.use_custom_key = true;
context->setSetting("distributed_group_by_no_merge", 2);
}
}

View File

@ -384,22 +384,22 @@ BlockIO InterpreterSystemQuery::execute()
{
auto caches = FileCacheFactory::instance().getAll();
for (const auto & [_, cache_data] : caches)
cache_data->cache->removeAllReleasable();
cache_data->cache->removeAllReleasable(FileCache::getCommonUser().user_id);
}
else
{
auto cache = FileCacheFactory::instance().getByName(query.filesystem_cache_name)->cache;
if (query.key_to_drop.empty())
{
cache->removeAllReleasable();
cache->removeAllReleasable(FileCache::getCommonUser().user_id);
}
else
{
auto key = FileCacheKey::fromKeyString(query.key_to_drop);
if (query.offset_to_drop.has_value())
cache->removeFileSegment(key, query.offset_to_drop.value());
cache->removeFileSegment(key, query.offset_to_drop.value(), FileCache::getCommonUser().user_id);
else
cache->removeKey(key);
cache->removeKey(key, FileCache::getCommonUser().user_id);
}
}
break;
@ -424,7 +424,9 @@ BlockIO InterpreterSystemQuery::execute()
for (const auto & file_segment : file_segments)
{
size_t i = 0;
const auto path = cache->getPathInLocalCache(file_segment.key, file_segment.offset, file_segment.kind);
const auto path = cache->getFileSegmentPath(
file_segment.key, file_segment.offset, file_segment.kind,
FileCache::UserInfo(file_segment.user_id, file_segment.user_weight));
res_columns[i++]->insert(cache_name);
res_columns[i++]->insert(path);
res_columns[i++]->insert(file_segment.downloaded_size);

View File

@ -105,7 +105,9 @@ FileSegmentsHolderPtr TemporaryDataOnDisk::createCacheFile(size_t max_file_size)
ProfileEvents::increment(ProfileEvents::ExternalProcessingFilesTotal);
const auto key = FileSegment::Key::random();
auto holder = file_cache->set(key, 0, std::max(10_MiB, max_file_size), CreateFileSegmentSettings(FileSegmentKind::Temporary, /* unbounded */ true));
auto holder = file_cache->set(
key, 0, std::max(10_MiB, max_file_size),
CreateFileSegmentSettings(FileSegmentKind::Temporary, /* unbounded */ true), FileCache::getCommonUser());
chassert(holder->size() == 1);
holder->back().getKeyMetadata()->createBaseDirectory();
@ -377,7 +379,7 @@ String TemporaryFileStream::getPath() const
if (file)
return file->getAbsolutePath();
if (segment_holder && !segment_holder->empty())
return segment_holder->front().getPathInLocalCache();
return segment_holder->front().getPath();
throw Exception(ErrorCodes::LOGICAL_ERROR, "TemporaryFileStream has no file");
}

View File

@ -20,6 +20,12 @@ namespace ErrorCodes
extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER;
}
bool canUseCustomKey(const Settings & settings, const Cluster & cluster, const Context & context)
{
return settings.max_parallel_replicas > 1 && context.getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY
&& cluster.getShardCount() == 1 && cluster.getShardsInfo()[0].getAllNodeCount() > 1;
}
ASTPtr getCustomKeyFilterForParallelReplica(
size_t replicas_count,
size_t replica_num,
@ -28,7 +34,7 @@ ASTPtr getCustomKeyFilterForParallelReplica(
const ColumnsDescription & columns,
const ContextPtr & context)
{
chassert(replicas_count > 1);
assert(replicas_count > 1);
if (filter_type == ParallelReplicasCustomKeyFilterType::DEFAULT)
{
// first we do modulo with replica count

View File

@ -9,6 +9,9 @@
namespace DB
{
bool canUseCustomKey(const Settings & settings, const Cluster & cluster, const Context & context);
/// Get AST for filter created from custom_key
/// replica_num is the number of the replica for which we are generating filter starting from 0
ASTPtr getCustomKeyFilterForParallelReplica(

View File

@ -1,14 +1,11 @@
#include <gtest/gtest.h>
#include <filesystem>
#include <iomanip>
#include <iostream>
#include <algorithm>
#include <numeric>
#include <random>
#include <memory>
#include <thread>
#include <Common/iota.h>
@ -49,6 +46,11 @@ using namespace DB;
static constexpr auto TEST_LOG_LEVEL = "debug";
namespace DB::ErrorCodes
{
extern const int FILECACHE_ACCESS_DENIED;
}
void printRanges(const auto & segments)
{
std::cerr << "\nHaving file segments: ";
@ -86,6 +88,7 @@ using HolderPtr = FileSegmentsHolderPtr;
fs::path caches_dir = fs::current_path() / "lru_cache_test";
std::string cache_base_path = caches_dir / "cache1" / "";
std::string cache_base_path2 = caches_dir / "cache2" / "";
std::string cache_base_path3 = caches_dir / "cache3" / "";
void assertEqual(const FileSegmentsHolderPtr & file_segments, const Ranges & expected_ranges, const States & expected_states = {})
@ -147,6 +150,18 @@ void assertEqual(const std::vector<FileSegment::Info> & file_segments, const Ran
}
}
void assertEqual(const IFileCachePriority::PriorityDumpPtr & dump, const Ranges & expected_ranges, const States & expected_states = {})
{
if (const auto * lru = dynamic_cast<const LRUFileCachePriority::LRUPriorityDump *>(dump.get()))
{
assertEqual(lru->infos, expected_ranges, expected_states);
}
else
{
ASSERT_TRUE(false);
}
}
void assertProtectedOrProbationary(const std::vector<FileSegmentInfo> & file_segments, const Ranges & expected, bool assert_protected)
{
std::cerr << "\nFile segments: ";
@ -190,6 +205,30 @@ void assertProbationary(const std::vector<FileSegmentInfo> & file_segments, cons
assertProtectedOrProbationary(file_segments, expected, false);
}
void assertProtected(const IFileCachePriority::PriorityDumpPtr & dump, const Ranges & expected)
{
if (const auto * lru = dynamic_cast<const LRUFileCachePriority::LRUPriorityDump *>(dump.get()))
{
assertProtected(lru->infos, expected);
}
else
{
ASSERT_TRUE(false);
}
}
void assertProbationary(const IFileCachePriority::PriorityDumpPtr & dump, const Ranges & expected)
{
if (const auto * lru = dynamic_cast<const LRUFileCachePriority::LRUPriorityDump *>(dump.get()))
{
assertProbationary(lru->infos, expected);
}
else
{
ASSERT_TRUE(false);
}
}
FileSegment & get(const HolderPtr & holder, int i)
{
auto it = std::next(holder->begin(), i);
@ -320,14 +359,20 @@ TEST_F(FileCacheTest, LRUPolicy)
const size_t file_size = INT_MAX; // the value doesn't really matter because boundary_alignment == 1.
const auto user = FileCache::getCommonUser();
{
std::cerr << "Step 1\n";
auto cache = DB::FileCache("1", settings);
cache.initialize();
auto key = cache.createKeyForPath("key1");
auto get_or_set = [&](size_t offset, size_t size)
{
auto holder = cache.getOrSet(key, 0, 10, file_size, {}); /// Add range [0, 9]
return cache.getOrSet(key, offset, size, file_size, {}, 0, user);
};
{
auto holder = get_or_set(0, 10); /// Add range [0, 9]
assertEqual(holder, { Range(0, 9) }, { State::EMPTY });
download(holder->front());
assertEqual(holder, { Range(0, 9) }, { State::DOWNLOADED });
@ -337,7 +382,7 @@ TEST_F(FileCacheTest, LRUPolicy)
/// Current cache: [__________]
/// ^ ^
/// 0 9
assertEqual(cache.getFileSegmentInfos(key), { Range(0, 9) });
assertEqual(cache.getFileSegmentInfos(key, user.user_id), { Range(0, 9) });
assertEqual(cache.dumpQueue(), { Range(0, 9) });
ASSERT_EQ(cache.getFileSegmentsNum(), 1);
ASSERT_EQ(cache.getUsedCacheSize(), 10);
@ -346,7 +391,7 @@ TEST_F(FileCacheTest, LRUPolicy)
{
/// Want range [5, 14], but [0, 9] already in cache, so only [10, 14] will be put in cache.
auto holder = cache.getOrSet(key, 5, 10, file_size, {});
auto holder = get_or_set(5, 10);
assertEqual(holder, { Range(0, 9), Range(10, 14) }, { State::DOWNLOADED, State::EMPTY });
download(get(holder, 1));
assertEqual(holder, { Range(0, 9), Range(10, 14) }, { State::DOWNLOADED, State::DOWNLOADED });
@ -356,7 +401,7 @@ TEST_F(FileCacheTest, LRUPolicy)
/// Current cache: [__________][_____]
/// ^ ^^ ^
/// 0 910 14
assertEqual(cache.getFileSegmentInfos(key), { Range(0, 9), Range(10, 14) });
assertEqual(cache.getFileSegmentInfos(key, user.user_id), { Range(0, 9), Range(10, 14) });
assertEqual(cache.dumpQueue(), { Range(0, 9), Range(10, 14) });
ASSERT_EQ(cache.getFileSegmentsNum(), 2);
ASSERT_EQ(cache.getUsedCacheSize(), 15);
@ -365,23 +410,23 @@ TEST_F(FileCacheTest, LRUPolicy)
/// Get [9, 9]
{
auto holder = cache.getOrSet(key, 9, 1, file_size, {});
auto holder = get_or_set(9, 1);
assertEqual(holder, { Range(0, 9) }, { State::DOWNLOADED });
increasePriority(holder);
}
assertEqual(cache.dumpQueue(), { Range(10, 14), Range(0, 9) });
/// Get [9, 10]
assertEqual(cache.getOrSet(key, 9, 2, file_size, {}), {Range(0, 9), Range(10, 14)}, {State::DOWNLOADED, State::DOWNLOADED});
assertEqual(get_or_set(9, 2), {Range(0, 9), Range(10, 14)}, {State::DOWNLOADED, State::DOWNLOADED});
/// Get [10, 10]
{
auto holder = cache.getOrSet(key, 10, 1, file_size, {});
auto holder = get_or_set(10, 1);
assertEqual(holder, { Range(10, 14) }, { State::DOWNLOADED });
increasePriority(holder);
}
assertEqual(cache.getFileSegmentInfos(key), { Range(0, 9), Range(10, 14) });
assertEqual(cache.getFileSegmentInfos(key, user.user_id), { Range(0, 9), Range(10, 14) });
assertEqual(cache.dumpQueue(), { Range(0, 9), Range(10, 14) });
ASSERT_EQ(cache.getFileSegmentsNum(), 2);
ASSERT_EQ(cache.getUsedCacheSize(), 15);
@ -389,19 +434,19 @@ TEST_F(FileCacheTest, LRUPolicy)
std::cerr << "Step 4\n";
{
auto holder = cache.getOrSet(key, 17, 4, file_size, {});
auto holder = get_or_set(17, 4);
download(holder); /// Get [17, 20]
increasePriority(holder);
}
{
auto holder = cache.getOrSet(key, 24, 3, file_size, {});
auto holder = get_or_set(24, 3);
download(holder); /// Get [24, 26]
increasePriority(holder);
}
{
auto holder = cache.getOrSet(key, 27, 1, file_size, {});
auto holder = get_or_set(27, 1);
download(holder); /// Get [27, 27]
increasePriority(holder);
}
@ -410,14 +455,14 @@ TEST_F(FileCacheTest, LRUPolicy)
/// ^ ^^ ^ ^ ^ ^ ^^^
/// 0 910 14 17 20 24 2627
///
assertEqual(cache.getFileSegmentInfos(key), { Range(0, 9), Range(10, 14), Range(17, 20), Range(24, 26), Range(27, 27) });
assertEqual(cache.getFileSegmentInfos(key, user.user_id), { Range(0, 9), Range(10, 14), Range(17, 20), Range(24, 26), Range(27, 27) });
assertEqual(cache.dumpQueue(), { Range(0, 9), Range(10, 14), Range(17, 20), Range(24, 26), Range(27, 27) });
ASSERT_EQ(cache.getFileSegmentsNum(), 5);
ASSERT_EQ(cache.getUsedCacheSize(), 23);
std::cerr << "Step 5\n";
{
auto holder = cache.getOrSet(key, 0, 26, file_size, {}); /// Get [0, 25]
auto holder = get_or_set(0, 26);
assertEqual(holder,
{ Range(0, 9), Range(10, 14), Range(15, 16), Range(17, 20), Range(21, 23), Range(24, 26) },
{ State::DOWNLOADED, State::DOWNLOADED, State::EMPTY, State::DOWNLOADED, State::EMPTY, State::DOWNLOADED });
@ -434,12 +479,12 @@ TEST_F(FileCacheTest, LRUPolicy)
/// Let's not invalidate pointers to returned segments from range [0, 25] and
/// as max elements size is reached, next attempt to put something in cache should fail.
/// This will also check that [27, 27] was indeed evicted.
auto holder2 = cache.getOrSet(key, 27, 1, file_size, {});
auto holder2 = get_or_set(27, 1);
assertEqual(holder2, { Range(27, 27) }, { State::EMPTY });
assertDownloadFails(holder2->front());
assertEqual(holder2, { Range(27, 27) }, { State::DETACHED });
auto holder3 = cache.getOrSet(key, 28, 3, file_size, {});
auto holder3 = get_or_set(28, 3);
assertEqual(holder3, { Range(28, 30) }, { State::EMPTY });
assertDownloadFails(holder3->front());
assertEqual(holder3, { Range(28, 30) }, { State::DETACHED });
@ -453,7 +498,7 @@ TEST_F(FileCacheTest, LRUPolicy)
/// ^ ^ ^
/// 0 20 24
///
assertEqual(cache.getFileSegmentInfos(key), { Range(0, 9), Range(10, 14), Range(15, 16), Range(17, 20), Range(24, 26) });
assertEqual(cache.getFileSegmentInfos(key, user.user_id), { Range(0, 9), Range(10, 14), Range(15, 16), Range(17, 20), Range(24, 26) });
assertEqual(cache.dumpQueue(), { Range(0, 9), Range(10, 14), Range(15, 16), Range(17, 20), Range(24, 26) });
ASSERT_EQ(cache.getFileSegmentsNum(), 5);
ASSERT_EQ(cache.getUsedCacheSize(), 24);
@ -461,7 +506,7 @@ TEST_F(FileCacheTest, LRUPolicy)
std::cerr << "Step 6\n";
{
auto holder = cache.getOrSet(key, 12, 10, file_size, {}); /// Get [12, 21]
auto holder = get_or_set(12, 10);
assertEqual(holder,
{ Range(10, 14), Range(15, 16), Range(17, 20), Range(21, 21) },
{ State::DOWNLOADED, State::DOWNLOADED, State::DOWNLOADED, State::EMPTY });
@ -475,14 +520,14 @@ TEST_F(FileCacheTest, LRUPolicy)
/// Current cache: [_____][__][____][_] [___]
/// ^ ^ ^ ^ ^
/// 10 17 21 24 26
assertEqual(cache.getFileSegmentInfos(key), { Range(10, 14), Range(15, 16), Range(17, 20), Range(21, 21), Range(24, 26) });
assertEqual(cache.getFileSegmentInfos(key, user.user_id), { Range(10, 14), Range(15, 16), Range(17, 20), Range(21, 21), Range(24, 26) });
assertEqual(cache.dumpQueue(), { Range(24, 26), Range(10, 14), Range(15, 16), Range(17, 20), Range(21, 21) });
ASSERT_EQ(cache.getFileSegmentsNum(), 5);
ASSERT_EQ(cache.getUsedCacheSize(), 15);
std::cerr << "Step 7\n";
{
auto holder = cache.getOrSet(key, 23, 5, file_size, {}); /// Get [23, 27]
auto holder = get_or_set(23, 5);
assertEqual(holder,
{ Range(23, 23), Range(24, 26), Range(27, 27) },
{ State::EMPTY, State::DOWNLOADED, State::EMPTY });
@ -494,32 +539,32 @@ TEST_F(FileCacheTest, LRUPolicy)
/// Current cache: [____][_] [][___][__]
/// ^ ^ ^^^ ^^ ^
/// 17 21 2324 26 27
assertEqual(cache.getFileSegmentInfos(key), { Range(17, 20), Range(21, 21), Range(23, 23), Range(24, 26), Range(27, 27) });
assertEqual(cache.getFileSegmentInfos(key, user.user_id), { Range(17, 20), Range(21, 21), Range(23, 23), Range(24, 26), Range(27, 27) });
assertEqual(cache.dumpQueue(), { Range(17, 20), Range(21, 21), Range(23, 23), Range(24, 26), Range(27, 27) });
ASSERT_EQ(cache.getFileSegmentsNum(), 5);
ASSERT_EQ(cache.getUsedCacheSize(), 10);
std::cerr << "Step 8\n";
{
auto holder = cache.getOrSet(key, 2, 3, file_size, {}); /// Get [2, 4]
auto holder = get_or_set(2, 3); /// Get [2, 4]
assertEqual(holder, { Range(2, 4) }, { State::EMPTY });
auto holder2 = cache.getOrSet(key, 30, 2, file_size, {}); /// Get [30, 31]
auto holder2 = get_or_set(30, 2); /// Get [30, 31]
assertEqual(holder2, { Range(30, 31) }, { State::EMPTY });
download(get(holder, 0));
download(get(holder2, 0));
auto holder3 = cache.getOrSet(key, 23, 1, file_size, {}); /// Get [23, 23]
auto holder3 = get_or_set(23, 1); /// Get [23, 23]
assertEqual(holder3, { Range(23, 23) }, { State::DOWNLOADED });
auto holder4 = cache.getOrSet(key, 24, 3, file_size, {}); /// Get [24, 26]
auto holder4 = get_or_set(24, 3); /// Get [24, 26]
assertEqual(holder4, { Range(24, 26) }, { State::DOWNLOADED });
auto holder5 = cache.getOrSet(key, 27, 1, file_size, {}); /// Get [27, 27]
auto holder5 = get_or_set(27, 1); /// Get [27, 27]
assertEqual(holder5, { Range(27, 27) }, { State::DOWNLOADED });
auto holder6 = cache.getOrSet(key, 0, 40, file_size, {});
auto holder6 = get_or_set(0, 40);
assertEqual(holder6,
{ Range(0, 1), Range(2, 4), Range(5, 22), Range(23, 23), Range(24, 26), Range(27, 27), Range(28, 29), Range(30, 31), Range(32, 39) },
{ State::EMPTY, State::DOWNLOADED, State::EMPTY, State::DOWNLOADED, State::DOWNLOADED, State::DOWNLOADED, State::EMPTY, State::DOWNLOADED, State::EMPTY });
@ -540,21 +585,21 @@ TEST_F(FileCacheTest, LRUPolicy)
/// Current cache: [___] [_][___][_] [__]
/// ^ ^ ^ ^ ^ ^ ^ ^
/// 2 4 23 24 26 27 30 31
assertEqual(cache.getFileSegmentInfos(key), { Range(2, 4), Range(23, 23), Range(24, 26), Range(27, 27), Range(30, 31) });
assertEqual(cache.getFileSegmentInfos(key, user.user_id), { Range(2, 4), Range(23, 23), Range(24, 26), Range(27, 27), Range(30, 31) });
assertEqual(cache.dumpQueue(), { Range(2, 4), Range(23, 23), Range(24, 26), Range(27, 27), Range(30, 31) });
std::cerr << "Step 9\n";
/// Get [2, 4]
{
auto holder = cache.getOrSet(key, 2, 3, file_size, {});
auto holder = get_or_set(2, 3);
assertEqual(holder, { Range(2, 4) }, { State::DOWNLOADED });
increasePriority(holder);
}
{
auto holder = cache.getOrSet(key, 25, 5, file_size, {}); /// Get [25, 29]
auto holder = get_or_set(25, 5); /// Get [25, 29]
assertEqual(holder,
{ Range(24, 26), Range(27, 27), Range(28, 29) },
{ State::DOWNLOADED, State::DOWNLOADED, State::EMPTY });
@ -576,7 +621,7 @@ TEST_F(FileCacheTest, LRUPolicy)
chassert(&DB::CurrentThread::get() == &thread_status_1);
DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1);
auto holder2 = cache.getOrSet(key, 25, 5, file_size, {}); /// Get [25, 29] once again.
auto holder2 = get_or_set(25, 5); /// Get [25, 29] once again.
assertEqual(holder2,
{ Range(24, 26), Range(27, 27), Range(28, 29) },
{ State::DOWNLOADED, State::DOWNLOADED, State::DOWNLOADING });
@ -611,7 +656,7 @@ TEST_F(FileCacheTest, LRUPolicy)
/// Current cache: [___] [___][_][__][__]
/// ^ ^ ^ ^ ^^ ^^ ^
/// 2 4 24 26 27 2930 31
assertEqual(cache.getFileSegmentInfos(key), { Range(2, 4), Range(24, 26), Range(27, 27), Range(28, 29), Range(30, 31) });
assertEqual(cache.getFileSegmentInfos(key, user.user_id), { Range(2, 4), Range(24, 26), Range(27, 27), Range(28, 29), Range(30, 31) });
assertEqual(cache.dumpQueue(), { Range(30, 31), Range(2, 4), Range(24, 26), Range(27, 27), Range(28, 29) });
std::cerr << "Step 10\n";
@ -620,7 +665,7 @@ TEST_F(FileCacheTest, LRUPolicy)
/// state is changed not manually via segment->completeWithState(state) but from destructor of holder
/// and notify_all() is also called from destructor of holder.
auto holder = cache.getOrSet(key, 3, 23, file_size, {}); /// Get [3, 25]
auto holder = get_or_set(3, 23); /// Get [3, 25]
assertEqual(holder,
{ Range(2, 4), Range(5, 23), Range(24, 26) },
{ State::DOWNLOADED, State::EMPTY, State::DOWNLOADED });
@ -642,7 +687,7 @@ TEST_F(FileCacheTest, LRUPolicy)
chassert(&DB::CurrentThread::get() == &thread_status_1);
DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1);
auto holder2 = cache.getOrSet(key, 3, 23, file_size, {}); /// Get [3, 25] once again
auto holder2 = get_or_set(3, 23); /// get [3, 25] once again.
assertEqual(holder,
{ Range(2, 4), Range(5, 23), Range(24, 26) },
{ State::DOWNLOADED, State::DOWNLOADING, State::DOWNLOADED });
@ -687,7 +732,7 @@ TEST_F(FileCacheTest, LRUPolicy)
/// Get [2, 29]
assertEqual(
cache2.getOrSet(key, 2, 28, file_size, {}),
cache2.getOrSet(key, 2, 28, file_size, {}, 0, user),
{Range(2, 4), Range(5, 23), Range(24, 26), Range(27, 27), Range(28, 29)},
{State::DOWNLOADED, State::DOWNLOADED, State::DOWNLOADED, State::DOWNLOADED, State::DOWNLOADED});
}
@ -706,7 +751,7 @@ TEST_F(FileCacheTest, LRUPolicy)
/// Get [0, 24]
assertEqual(
cache2.getOrSet(key, 0, 25, file_size, {}),
cache2.getOrSet(key, 0, 25, file_size, {}, 0, user),
{Range(0, 9), Range(10, 19), Range(20, 24)},
{State::EMPTY, State::EMPTY, State::EMPTY});
}
@ -718,21 +763,21 @@ TEST_F(FileCacheTest, LRUPolicy)
auto cache = FileCache("4", settings);
cache.initialize();
const auto key = cache.createKeyForPath("key10");
const auto key_path = cache.getPathInLocalCache(key);
const auto key_path = cache.getKeyPath(key, user);
cache.removeAllReleasable();
cache.removeAllReleasable(user.user_id);
ASSERT_EQ(cache.getUsedCacheSize(), 0);
ASSERT_TRUE(!fs::exists(key_path));
ASSERT_TRUE(!fs::exists(fs::path(key_path).parent_path()));
download(cache.getOrSet(key, 0, 10, file_size, {}));
download(cache.getOrSet(key, 0, 10, file_size, {}, 0, user));
ASSERT_EQ(cache.getUsedCacheSize(), 10);
ASSERT_TRUE(fs::exists(cache.getPathInLocalCache(key, 0, FileSegmentKind::Regular)));
ASSERT_TRUE(fs::exists(cache.getFileSegmentPath(key, 0, FileSegmentKind::Regular, user)));
cache.removeAllReleasable();
cache.removeAllReleasable(user.user_id);
ASSERT_EQ(cache.getUsedCacheSize(), 0);
ASSERT_TRUE(!fs::exists(key_path));
ASSERT_TRUE(!fs::exists(cache.getPathInLocalCache(key, 0, FileSegmentKind::Regular)));
ASSERT_TRUE(!fs::exists(cache.getFileSegmentPath(key, 0, FileSegmentKind::Regular, user)));
}
std::cerr << "Step 14\n";
@ -742,18 +787,18 @@ TEST_F(FileCacheTest, LRUPolicy)
auto cache = DB::FileCache("5", settings);
cache.initialize();
const auto key = cache.createKeyForPath("key10");
const auto key_path = cache.getPathInLocalCache(key);
const auto key_path = cache.getKeyPath(key, user);
cache.removeAllReleasable();
cache.removeAllReleasable(user.user_id);
ASSERT_EQ(cache.getUsedCacheSize(), 0);
ASSERT_TRUE(!fs::exists(key_path));
ASSERT_TRUE(!fs::exists(fs::path(key_path).parent_path()));
download(cache.getOrSet(key, 0, 10, file_size, {}));
download(cache.getOrSet(key, 0, 10, file_size, {}, 0, user));
ASSERT_EQ(cache.getUsedCacheSize(), 10);
ASSERT_TRUE(fs::exists(key_path));
cache.removeAllReleasable();
cache.removeAllReleasable(user.user_id);
ASSERT_EQ(cache.getUsedCacheSize(), 0);
sleepForSeconds(2);
ASSERT_TRUE(!fs::exists(key_path));
@ -770,17 +815,18 @@ TEST_F(FileCacheTest, writeBuffer)
FileCache cache("6", settings);
cache.initialize();
const auto user = FileCache::getCommonUser();
auto write_to_cache = [&cache, this](const String & key, const Strings & data, bool flush, ReadBufferPtr * out_read_buffer = nullptr)
auto write_to_cache = [&, this](const String & key, const Strings & data, bool flush, ReadBufferPtr * out_read_buffer = nullptr)
{
CreateFileSegmentSettings segment_settings;
segment_settings.kind = FileSegmentKind::Temporary;
segment_settings.unbounded = true;
auto cache_key = cache.createKeyForPath(key);
auto holder = cache.set(cache_key, 0, 3, segment_settings);
auto holder = cache.set(cache_key, 0, 3, segment_settings, user);
/// The same is done in TemporaryDataOnDisk::createStreamToCacheFile.
std::filesystem::create_directories(cache.getPathInLocalCache(cache_key));
std::filesystem::create_directories(cache.getKeyPath(cache_key, user));
EXPECT_EQ(holder->size(), 1);
auto & segment = holder->front();
WriteBufferToFileSegment out(&segment);
@ -818,7 +864,7 @@ TEST_F(FileCacheTest, writeBuffer)
std::vector<fs::path> file_segment_paths;
{
auto holder = write_to_cache("key1", {"abc", "defg"}, false);
file_segment_paths.emplace_back(holder->front().getPathInLocalCache());
file_segment_paths.emplace_back(holder->front().getPath());
ASSERT_EQ(fs::file_size(file_segment_paths.back()), 7);
EXPECT_EQ(holder->front().range().size(), 7);
@ -829,7 +875,7 @@ TEST_F(FileCacheTest, writeBuffer)
ReadBufferPtr reader = nullptr;
auto holder2 = write_to_cache("key2", {"22", "333", "4444", "55555", "1"}, true, &reader);
file_segment_paths.emplace_back(holder2->front().getPathInLocalCache());
file_segment_paths.emplace_back(holder2->front().getPath());
std::cerr << "\nFile segments: " << holder2->toString() << "\n";
@ -900,9 +946,10 @@ TEST_F(FileCacheTest, temporaryData)
DB::FileCache file_cache("7", settings);
file_cache.initialize();
const auto user = FileCache::getCommonUser();
auto tmp_data_scope = std::make_shared<TemporaryDataOnDiskScope>(nullptr, &file_cache, 0);
auto some_data_holder = file_cache.getOrSet(file_cache.createKeyForPath("some_data"), 0, 5_KiB, 5_KiB, CreateFileSegmentSettings{});
auto some_data_holder = file_cache.getOrSet(file_cache.createKeyForPath("some_data"), 0, 5_KiB, 5_KiB, CreateFileSegmentSettings{}, 0, user);
{
ASSERT_EQ(some_data_holder->size(), 5);
@ -1041,10 +1088,11 @@ TEST_F(FileCacheTest, CachedReadBuffer)
auto cache = std::make_shared<DB::FileCache>("8", settings);
cache->initialize();
auto key = cache->createKeyForPath(file_path);
const auto user = FileCache::getCommonUser();
{
auto cached_buffer = std::make_shared<CachedOnDiskReadBufferFromFile>(
file_path, key, cache, read_buffer_creator, read_settings, "test", s.size(), false, false, std::nullopt, nullptr);
file_path, key, cache, user, read_buffer_creator, read_settings, "test", s.size(), false, false, std::nullopt, nullptr);
WriteBufferFromOwnString result;
copyData(*cached_buffer, result);
@ -1059,7 +1107,7 @@ TEST_F(FileCacheTest, CachedReadBuffer)
modified_settings.remote_fs_buffer_size = 10;
auto cached_buffer = std::make_shared<CachedOnDiskReadBufferFromFile>(
file_path, key, cache, read_buffer_creator, modified_settings, "test", s.size(), false, false, std::nullopt, nullptr);
file_path, key, cache, user, read_buffer_creator, modified_settings, "test", s.size(), false, false, std::nullopt, nullptr);
cached_buffer->next();
assertEqual(cache->dumpQueue(), {Range(10, 14), Range(15, 19), Range(20, 24), Range(25, 29), Range(0, 4), Range(5, 9)});
@ -1146,6 +1194,7 @@ TEST_F(FileCacheTest, SLRUPolicy)
const size_t file_size = -1; // the value doesn't really matter because boundary_alignment == 1.
size_t file_cache_name = 0;
const auto user = FileCache::getCommonUser();
{
auto cache = DB::FileCache(std::to_string(++file_cache_name), settings);
@ -1156,7 +1205,7 @@ TEST_F(FileCacheTest, SLRUPolicy)
{
std::cerr << "Add [" << offset << ", " << offset + size - 1 << "]" << std::endl;
auto holder = cache.getOrSet(key, offset, size, file_size, {});
auto holder = cache.getOrSet(key, offset, size, file_size, {}, 0, user);
assertEqual(holder, { Range(offset, offset + size - 1) }, { State::EMPTY });
download(holder->front());
assertEqual(holder, { Range(offset, offset + size - 1) }, { State::DOWNLOADED });
@ -1164,7 +1213,7 @@ TEST_F(FileCacheTest, SLRUPolicy)
auto check_covering_range = [&](size_t offset, size_t size, Ranges covering_ranges)
{
auto holder = cache.getOrSet(key, offset, size, file_size, {});
auto holder = cache.getOrSet(key, offset, size, file_size, {}, 0, user);
std::vector<State> states(covering_ranges.size(), State::DOWNLOADED);
assertEqual(holder, covering_ranges, states);
increasePriority(holder);
@ -1173,7 +1222,7 @@ TEST_F(FileCacheTest, SLRUPolicy)
add_range(0, 10);
add_range(10, 5);
assertEqual(cache.getFileSegmentInfos(key), { Range(0, 9), Range(10, 14) });
assertEqual(cache.getFileSegmentInfos(key, user.user_id), { Range(0, 9), Range(10, 14) });
assertEqual(cache.dumpQueue(), { Range(0, 9), Range(10, 14) });
ASSERT_EQ(cache.getFileSegmentsNum(), 2);
@ -1203,7 +1252,7 @@ TEST_F(FileCacheTest, SLRUPolicy)
assertProbationary(cache.dumpQueue(), { Range(17, 20), Range(24, 26), Range(27, 27) });
assertProtected(cache.dumpQueue(), { Range(0, 9), Range(10, 14) });
assertEqual(cache.getFileSegmentInfos(key), { Range(0, 9), Range(10, 14), Range(17, 20), Range(24, 26), Range(27, 27) });
assertEqual(cache.getFileSegmentInfos(key, user.user_id), { Range(0, 9), Range(10, 14), Range(17, 20), Range(24, 26), Range(27, 27) });
ASSERT_EQ(cache.getFileSegmentsNum(), 5);
ASSERT_EQ(cache.getUsedCacheSize(), 23);
@ -1223,7 +1272,7 @@ TEST_F(FileCacheTest, SLRUPolicy)
assertProbationary(cache.dumpQueue(), { Range(24, 26), Range(10, 14) });
assertProtected(cache.dumpQueue(), { Range(0, 9), Range(27, 27), Range(28, 30) });
assertEqual(cache.getFileSegmentInfos(key), { Range(0, 9), Range(10, 14), Range(24, 26), Range(27, 27), Range(28, 30) });
assertEqual(cache.getFileSegmentInfos(key, user.user_id), { Range(0, 9), Range(10, 14), Range(24, 26), Range(27, 27), Range(28, 30) });
ASSERT_EQ(cache.getFileSegmentsNum(), 5);
ASSERT_EQ(cache.getUsedCacheSize(), 22);
}
@ -1263,7 +1312,7 @@ TEST_F(FileCacheTest, SLRUPolicy)
};
auto cached_buffer = std::make_shared<CachedOnDiskReadBufferFromFile>(
file, key, cache, read_buffer_creator, read_settings, "test", expect_result.size(), false, false, std::nullopt, nullptr);
file, key, cache, user, read_buffer_creator, read_settings, "test", expect_result.size(), false, false, std::nullopt, nullptr);
WriteBufferFromOwnString result;
copyData(*cached_buffer, result);
@ -1295,11 +1344,12 @@ TEST_F(FileCacheTest, SLRUPolicy)
auto dump = cache->dumpQueue();
assertEqual(dump, { Range(0, 4), Range(5, 9), Range(0, 4), Range(5, 9), Range(10, 14) });
ASSERT_EQ(dump[0].key, key2);
ASSERT_EQ(dump[1].key, key2);
ASSERT_EQ(dump[2].key, key1);
ASSERT_EQ(dump[3].key, key1);
ASSERT_EQ(dump[4].key, key1);
const auto & infos = dynamic_cast<const LRUFileCachePriority::LRUPriorityDump *>(dump.get())->infos;
ASSERT_EQ(infos[0].key, key2);
ASSERT_EQ(infos[1].key, key2);
ASSERT_EQ(infos[2].key, key1);
ASSERT_EQ(infos[3].key, key1);
ASSERT_EQ(infos[4].key, key1);
assertProbationary(cache->dumpQueue(), { Range(0, 4), Range(5, 9) });
assertProtected(cache->dumpQueue(), { Range(0, 4), Range(5, 9), Range(10, 14) });
@ -1309,11 +1359,12 @@ TEST_F(FileCacheTest, SLRUPolicy)
dump = cache->dumpQueue();
assertEqual(dump, { Range(0, 4), Range(5, 9), Range(10, 14), Range(0, 4), Range(5, 9) });
ASSERT_EQ(dump[0].key, key1);
ASSERT_EQ(dump[1].key, key1);
ASSERT_EQ(dump[2].key, key1);
ASSERT_EQ(dump[3].key, key2);
ASSERT_EQ(dump[4].key, key2);
const auto & infos2 = dynamic_cast<const LRUFileCachePriority::LRUPriorityDump *>(dump.get())->infos;
ASSERT_EQ(infos2[0].key, key1);
ASSERT_EQ(infos2[1].key, key1);
ASSERT_EQ(infos2[2].key, key1);
ASSERT_EQ(infos2[3].key, key2);
ASSERT_EQ(infos2[4].key, key2);
assertProbationary(cache->dumpQueue(), { Range(0, 4), Range(5, 9) });
assertProtected(cache->dumpQueue(), { Range(10, 14), Range(0, 4), Range(5, 9) });

View File

@ -1390,7 +1390,7 @@ void Planner::buildPlanForQueryNode()
}
}
if (query_context->canUseTaskBasedParallelReplicas() || !settings.parallel_replicas_custom_key.value.empty())
if (!settings.parallel_replicas_custom_key.value.empty())
{
/// Check support for JOIN for parallel replicas with custom key
if (planner_context->getTableExpressionNodeToData().size() > 1)

View File

@ -809,8 +809,9 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
else
{
if (auto * distributed = typeid_cast<StorageDistributed *>(storage.get());
distributed && query_context->canUseParallelReplicasCustomKey(*distributed->getCluster()))
distributed && canUseCustomKey(settings, *distributed->getCluster(), *query_context))
{
table_expression_query_info.use_custom_key = true;
planner_context->getMutableQueryContext()->setSetting("distributed_group_by_no_merge", 2);
}
}

View File

@ -18,7 +18,6 @@
#include <Client/ConnectionPool.h>
#include <Client/ConnectionPoolWithFailover.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Parsers/ASTFunction.h>
#include <boost/algorithm/string/join.hpp>
@ -232,6 +231,8 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact
add_extremes = context->getSettingsRef().extremes;
}
String query_string = formattedAST(shard.query);
scalars["_shard_num"]
= Block{{DataTypeUInt32().createColumnConst(1, shard.shard_info.shard_num), std::make_shared<DataTypeUInt32>(), "_shard_num"}};
@ -253,81 +254,29 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact
context->setSetting("cluster_for_parallel_replicas", cluster_name);
}
/// parallel replicas custom key case
if (shard.shard_filter_generator)
auto remote_query_executor = std::make_shared<RemoteQueryExecutor>(
shard.shard_info.pool, query_string, output_stream->header, context, throttler, scalars, external_tables, stage);
remote_query_executor->setLogger(log);
if (context->canUseTaskBasedParallelReplicas())
{
for (size_t i = 0; i < shard.shard_info.per_replica_pools.size(); ++i)
{
auto query = shard.query->clone();
auto & select_query = query->as<ASTSelectQuery &>();
auto shard_filter = shard.shard_filter_generator(i + 1);
if (shard_filter)
{
auto where_expression = select_query.where();
if (where_expression)
shard_filter = makeASTFunction("and", where_expression, shard_filter);
select_query.setExpression(ASTSelectQuery::Expression::WHERE, std::move(shard_filter));
}
const String query_string = formattedAST(query);
if (!priority_func_factory.has_value())
priority_func_factory = GetPriorityForLoadBalancing(LoadBalancing::ROUND_ROBIN, randomSeed());
GetPriorityForLoadBalancing::Func priority_func
= priority_func_factory->getPriorityFunc(LoadBalancing::ROUND_ROBIN, 0, shard.shard_info.pool->getPoolSize());
auto remote_query_executor = std::make_shared<RemoteQueryExecutor>(
shard.shard_info.pool,
query_string,
output_stream->header,
context,
throttler,
scalars,
external_tables,
stage,
std::nullopt,
priority_func);
remote_query_executor->setLogger(log);
remote_query_executor->setPoolMode(PoolMode::GET_ONE);
if (!table_func_ptr)
remote_query_executor->setMainTable(shard.main_table ? shard.main_table : main_table);
pipes.emplace_back(
createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes, async_read, async_query_sending));
addConvertingActions(pipes.back(), output_stream->header);
}
// when doing parallel reading from replicas (ParallelReplicasMode::READ_TASKS) on a shard:
// establish a connection to a replica on the shard, the replica will instantiate coordinator to manage parallel reading from replicas on the shard.
// The coordinator will return query result from the shard.
// Only one coordinator per shard is necessary. Therefore using PoolMode::GET_ONE to establish only one connection per shard.
// Using PoolMode::GET_MANY for this mode will(can) lead to instantiation of several coordinators (depends on max_parallel_replicas setting)
// each will execute parallel reading from replicas, so the query result will be multiplied by the number of created coordinators
remote_query_executor->setPoolMode(PoolMode::GET_ONE);
}
else
{
const String query_string = formattedAST(shard.query);
remote_query_executor->setPoolMode(PoolMode::GET_MANY);
auto remote_query_executor = std::make_shared<RemoteQueryExecutor>(
shard.shard_info.pool, query_string, output_stream->header, context, throttler, scalars, external_tables, stage);
remote_query_executor->setLogger(log);
if (!table_func_ptr)
remote_query_executor->setMainTable(shard.main_table ? shard.main_table : main_table);
if (context->canUseTaskBasedParallelReplicas())
{
// when doing parallel reading from replicas (ParallelReplicasMode::READ_TASKS) on a shard:
// establish a connection to a replica on the shard, the replica will instantiate coordinator to manage parallel reading from replicas on the shard.
// The coordinator will return query result from the shard.
// Only one coordinator per shard is necessary. Therefore using PoolMode::GET_ONE to establish only one connection per shard.
// Using PoolMode::GET_MANY for this mode will(can) lead to instantiation of several coordinators (depends on max_parallel_replicas setting)
// each will execute parallel reading from replicas, so the query result will be multiplied by the number of created coordinators
remote_query_executor->setPoolMode(PoolMode::GET_ONE);
}
else
remote_query_executor->setPoolMode(PoolMode::GET_MANY);
if (!table_func_ptr)
remote_query_executor->setMainTable(shard.main_table ? shard.main_table : main_table);
pipes.emplace_back(
createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes, async_read, async_query_sending));
addConvertingActions(pipes.back(), output_stream->header);
}
pipes.emplace_back(
createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes, async_read, async_query_sending));
addConvertingActions(pipes.back(), output_stream->header);
}
void ReadFromRemote::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
@ -357,7 +306,6 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep(
ParallelReplicasReadingCoordinatorPtr coordinator_,
Block header_,
QueryProcessingStage::Enum stage_,
StorageID main_table_,
ContextMutablePtr context_,
ThrottlerPtr throttler_,
Scalars scalars_,
@ -369,7 +317,6 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep(
, query_ast(query_ast_)
, coordinator(std::move(coordinator_))
, stage(std::move(stage_))
, main_table(std::move(main_table_))
, context(context_)
, throttler(throttler_)
, scalars(scalars_)

View File

@ -60,7 +60,6 @@ private:
Poco::Logger * log;
UInt32 shard_count;
const String cluster_name;
std::optional<GetPriorityForLoadBalancing> priority_func_factory;
void addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStreamFactory::Shard & shard);
void addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFactory::Shard & shard);
@ -76,7 +75,6 @@ public:
ParallelReplicasReadingCoordinatorPtr coordinator_,
Block header_,
QueryProcessingStage::Enum stage_,
StorageID main_table_,
ContextMutablePtr context_,
ThrottlerPtr throttler_,
Scalars scalars_,
@ -99,7 +97,6 @@ private:
ASTPtr query_ast;
ParallelReplicasReadingCoordinatorPtr coordinator;
QueryProcessingStage::Enum stage;
StorageID main_table;
ContextMutablePtr context;
ThrottlerPtr throttler;
Scalars scalars;

View File

@ -43,24 +43,13 @@ namespace ErrorCodes
}
RemoteQueryExecutor::RemoteQueryExecutor(
const String & query_,
const Block & header_,
ContextPtr context_,
const Scalars & scalars_,
const Tables & external_tables_,
QueryProcessingStage::Enum stage_,
std::optional<Extension> extension_,
GetPriorityForLoadBalancing::Func priority_func_)
: header(header_)
, query(query_)
, context(context_)
, scalars(scalars_)
, external_tables(external_tables_)
, stage(stage_)
const String & query_, const Block & header_, ContextPtr context_,
const Scalars & scalars_, const Tables & external_tables_,
QueryProcessingStage::Enum stage_, std::optional<Extension> extension_)
: header(header_), query(query_), context(context_), scalars(scalars_)
, external_tables(external_tables_), stage(stage_)
, extension(extension_)
, priority_func(priority_func_)
{
}
{}
RemoteQueryExecutor::RemoteQueryExecutor(
Connection & connection,
@ -111,16 +100,10 @@ RemoteQueryExecutor::RemoteQueryExecutor(
RemoteQueryExecutor::RemoteQueryExecutor(
const ConnectionPoolWithFailoverPtr & pool,
const String & query_,
const Block & header_,
ContextPtr context_,
const ThrottlerPtr & throttler,
const Scalars & scalars_,
const Tables & external_tables_,
QueryProcessingStage::Enum stage_,
std::optional<Extension> extension_,
GetPriorityForLoadBalancing::Func priority_func_)
: RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, extension_, priority_func_)
const String & query_, const Block & header_, ContextPtr context_,
const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_,
QueryProcessingStage::Enum stage_, std::optional<Extension> extension_)
: RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, extension_)
{
create_connections = [this, pool, throttler](AsyncCallback async_callback)->std::unique_ptr<IConnections>
{
@ -134,8 +117,7 @@ RemoteQueryExecutor::RemoteQueryExecutor(
if (main_table)
table_to_check = std::make_shared<QualifiedTableName>(main_table.getQualifiedName());
auto res = std::make_unique<HedgedConnections>(
pool, context, timeouts, throttler, pool_mode, table_to_check, std::move(async_callback), priority_func);
auto res = std::make_unique<HedgedConnections>(pool, context, timeouts, throttler, pool_mode, table_to_check, std::move(async_callback));
if (extension && extension->replica_info)
res->setReplicaInfo(*extension->replica_info);
return res;
@ -155,16 +137,14 @@ RemoteQueryExecutor::RemoteQueryExecutor(
pool_mode,
main_table.getQualifiedName(),
std::move(async_callback),
skip_unavailable_endpoints,
priority_func);
skip_unavailable_endpoints);
connection_entries.reserve(try_results.size());
for (auto & try_result : try_results)
connection_entries.emplace_back(std::move(try_result.entry));
}
else
{
connection_entries = pool->getMany(
timeouts, current_settings, pool_mode, std::move(async_callback), skip_unavailable_endpoints, priority_func);
connection_entries = pool->getMany(timeouts, current_settings, pool_mode, std::move(async_callback), skip_unavailable_endpoints);
}
auto res = std::make_unique<MultiplexedConnections>(std::move(connection_entries), current_settings, throttler);

View File

@ -50,7 +50,6 @@ public:
std::shared_ptr<TaskIterator> task_iterator = nullptr;
std::shared_ptr<ParallelReplicasReadingCoordinator> parallel_reading_coordinator = nullptr;
std::optional<IConnections::ReplicaInfo> replica_info = {};
GetPriorityForLoadBalancing::Func priority_func;
};
/// Takes already set connection.
@ -77,15 +76,9 @@ public:
/// Takes a pool and gets one or several connections from it.
RemoteQueryExecutor(
const ConnectionPoolWithFailoverPtr & pool,
const String & query_,
const Block & header_,
ContextPtr context_,
const ThrottlerPtr & throttler = nullptr,
const Scalars & scalars_ = Scalars(),
const Tables & external_tables_ = Tables(),
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete,
std::optional<Extension> extension_ = std::nullopt,
GetPriorityForLoadBalancing::Func priority_func = {});
const String & query_, const Block & header_, ContextPtr context_,
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete, std::optional<Extension> extension_ = std::nullopt);
~RemoteQueryExecutor();
@ -198,14 +191,9 @@ public:
private:
RemoteQueryExecutor(
const String & query_,
const Block & header_,
ContextPtr context_,
const Scalars & scalars_,
const Tables & external_tables_,
QueryProcessingStage::Enum stage_,
std::optional<Extension> extension_,
GetPriorityForLoadBalancing::Func priority_func = {});
const String & query_, const Block & header_, ContextPtr context_,
const Scalars & scalars_, const Tables & external_tables_,
QueryProcessingStage::Enum stage_, std::optional<Extension> extension_);
Block header;
Block totals;
@ -285,8 +273,6 @@ private:
Poco::Logger * log = nullptr;
GetPriorityForLoadBalancing::Func priority_func;
/// Send all scalars to remote servers
void sendScalars();

View File

@ -61,6 +61,8 @@ namespace ErrorCodes
static const size_t PACKET_HEADER_SIZE = 4;
static const size_t SSL_REQUEST_PAYLOAD_SIZE = 32;
static String showWarningsReplacementQuery(const String & query);
static String showCountWarningsReplacementQuery(const String & query);
static String selectEmptyReplacementQuery(const String & query);
static String showTableStatusReplacementQuery(const String & query);
static String killConnectionIdReplacementQuery(const String & query);
@ -86,6 +88,8 @@ MySQLHandler::MySQLHandler(
if (ssl_enabled)
server_capabilities |= CLIENT_SSL;
replacements.emplace("SHOW WARNINGS", showWarningsReplacementQuery);
replacements.emplace("SHOW COUNT(*) WARNINGS", showCountWarningsReplacementQuery);
replacements.emplace("KILL QUERY", killConnectionIdReplacementQuery);
replacements.emplace("SHOW TABLE STATUS LIKE", showTableStatusReplacementQuery);
replacements.emplace("SHOW VARIABLES", selectEmptyReplacementQuery);
@ -544,6 +548,18 @@ static bool isFederatedServerSetupSetCommand(const String & query)
return re2::RE2::FullMatch(query, expr);
}
/// Always return an empty set with appropriate column definitions for SHOW WARNINGS queries
/// See also: https://dev.mysql.com/doc/refman/8.0/en/show-warnings.html
static String showWarningsReplacementQuery([[maybe_unused]] const String & query)
{
return "SELECT '' AS Level, 0::UInt32 AS Code, '' AS Message WHERE false";
}
static String showCountWarningsReplacementQuery([[maybe_unused]] const String & query)
{
return "SELECT 0::UInt64 AS `@@session.warning_count`";
}
/// Replace "[query(such as SHOW VARIABLES...)]" into "".
static String selectEmptyReplacementQuery(const String & query)
{

View File

@ -1,5 +1,7 @@
#include <Storages/MergeTree/MergeTreeIndexSet.h>
#include <DataTypes/IDataType.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/TreeRewriter.h>
@ -13,7 +15,6 @@
#include <Planner/PlannerActionsVisitor.h>
#include <Storages/MergeTree/MergeTreeIndexUtils.h>
#include "DataTypes/IDataType.h"
namespace DB
{

View File

@ -348,7 +348,7 @@ IMergeTreeDataPart::Checksums checkDataPart(
if (!data_part_storage.isDirectory(file_name))
{
auto remote_path = data_part_storage.getRemotePath(file_name);
cache.removePathIfExists(remote_path);
cache.removePathIfExists(remote_path, FileCache::getCommonUser().user_id);
}
}

View File

@ -176,6 +176,8 @@ struct SelectQueryInfo
///
/// Configured in StorageDistributed::getQueryProcessingStage()
ClusterPtr optimized_cluster;
/// should we use custom key with the cluster
bool use_custom_key = false;
TreeRewriterResultPtr syntax_analyzer_result;

View File

@ -429,10 +429,15 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(
size_t nodes = getClusterQueriedNodes(settings, cluster);
query_info.cluster = cluster;
if (!local_context->canUseParallelReplicasCustomKey(*cluster))
if (query_info.use_custom_key)
{
LOG_INFO(log, "Single shard cluster used with custom_key, transforming replicas into virtual shards");
query_info.cluster = cluster->getClusterWithReplicasAsShards(settings, settings.max_parallel_replicas);
}
else
{
query_info.cluster = cluster;
if (nodes > 1 && settings.optimize_skip_unused_shards)
{
/// Always calculate optimized cluster here, to avoid conditions during read()
@ -875,22 +880,30 @@ void StorageDistributed::read(
storage_snapshot,
processed_stage);
const auto & settings = local_context->getSettingsRef();
auto settings = local_context->getSettingsRef();
ClusterProxy::AdditionalShardFilterGenerator additional_shard_filter_generator;
if (local_context->canUseParallelReplicasCustomKey(*query_info.getCluster()))
if (query_info.use_custom_key)
{
if (auto custom_key_ast = parseCustomKeyForTable(settings.parallel_replicas_custom_key, *local_context))
{
if (query_info.getCluster()->getShardCount() == 1)
{
// we are reading from single shard with multiple replicas but didn't transform replicas
// into virtual shards with custom_key set
throw Exception(ErrorCodes::LOGICAL_ERROR, "Replicas weren't transformed into virtual shards");
}
additional_shard_filter_generator =
[my_custom_key_ast = std::move(custom_key_ast),
column_description = this->getInMemoryMetadataPtr()->columns,
custom_key_type = settings.parallel_replicas_custom_key_filter_type.value,
context = local_context,
replica_count = query_info.getCluster()->getShardsInfo().front().per_replica_pools.size()](uint64_t replica_num) -> ASTPtr
[&, my_custom_key_ast = std::move(custom_key_ast), shard_count = query_info.cluster->getShardCount()](uint64_t shard_num) -> ASTPtr
{
return getCustomKeyFilterForParallelReplica(
replica_count, replica_num - 1, my_custom_key_ast, custom_key_type, column_description, context);
shard_count,
shard_num - 1,
my_custom_key_ast,
settings.parallel_replicas_custom_key_filter_type,
this->getInMemoryMetadataPtr()->columns,
local_context);
};
}
}

View File

@ -29,6 +29,7 @@
#include <Parsers/ASTSetQuery.h>
#include <Parsers/queryToString.h>
#include <Parsers/formatAST.h>
#include <Planner/Utils.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/ActiveDataPartSet.h>
#include <Storages/AlterCommands.h>
@ -39,6 +40,7 @@
#include <Storages/MergeTree/PartitionPruner.h>
#include <Storages/MergeTree/MergeList.h>
#include <Storages/MergeTree/checkDataPart.h>
#include <Storages/buildQueryTreeForShard.h>
#include <QueryPipeline/Pipe.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
@ -213,16 +215,25 @@ void StorageMergeTree::read(
{
if (local_context->canUseParallelReplicasOnInitiator() && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree)
{
const auto table_id = getStorageID();
const auto & modified_query_ast = ClusterProxy::rewriteSelectQuery(
local_context, query_info.query,
table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr);
ASTPtr modified_query_ast;
Block header;
if (local_context->getSettingsRef().allow_experimental_analyzer)
header = InterpreterSelectQueryAnalyzer::getSampleBlock(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze());
{
QueryTreeNodePtr modified_query_tree = query_info.query_tree->clone();
rewriteJoinToGlobalJoin(modified_query_tree);
modified_query_tree = buildQueryTreeForShard(query_info, modified_query_tree);
header = InterpreterSelectQueryAnalyzer::getSampleBlock(
modified_query_tree, local_context, SelectQueryOptions(processed_stage).analyze());
modified_query_ast = queryNodeToSelectQuery(modified_query_tree);
}
else
header = InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock();
{
const auto table_id = getStorageID();
modified_query_ast = ClusterProxy::rewriteSelectQuery(local_context, query_info.query,
table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr);
header
= InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock();
}
ClusterProxy::SelectStreamFactory select_stream_factory =
ClusterProxy::SelectStreamFactory(
@ -233,7 +244,6 @@ void StorageMergeTree::read(
ClusterProxy::executeQueryWithParallelReplicas(
query_plan,
getStorageID(),
select_stream_factory,
modified_query_ast,
local_context,

View File

@ -5385,7 +5385,9 @@ void StorageReplicatedMergeTree::readParallelReplicasImpl(
if (local_context->getSettingsRef().allow_experimental_analyzer)
{
auto modified_query_tree = buildQueryTreeForShard(query_info, query_info.query_tree);
QueryTreeNodePtr modified_query_tree = query_info.query_tree->clone();
rewriteJoinToGlobalJoin(modified_query_tree);
modified_query_tree = buildQueryTreeForShard(query_info, modified_query_tree);
header = InterpreterSelectQueryAnalyzer::getSampleBlock(
modified_query_tree, local_context, SelectQueryOptions(processed_stage).analyze());
@ -5408,7 +5410,6 @@ void StorageReplicatedMergeTree::readParallelReplicasImpl(
ClusterProxy::executeQueryWithParallelReplicas(
query_plan,
getStorageID(),
select_stream_factory,
modified_query_ast,
local_context,

View File

@ -30,6 +30,7 @@ ColumnsDescription StorageSystemFilesystemCache::getColumnsDescription()
{"downloaded_size", std::make_shared<DataTypeUInt64>()},
{"kind", std::make_shared<DataTypeString>()},
{"unbound", std::make_shared<DataTypeNumber<UInt8>>()},
{"user_id", std::make_shared<DataTypeString>()},
{"file_size", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>())},
};
}
@ -52,10 +53,12 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex
res_columns[i++]->insert(cache_name);
res_columns[i++]->insert(cache->getBasePath());
/// Do not use `file_segment->getPathInLocalCache` here because it will lead to nullptr dereference
/// Do not use `file_segment->getPath` here because it will lead to nullptr dereference
/// (because file_segments in getSnapshot doesn't have `cache` field set)
const auto path = cache->getPathInLocalCache(file_segment.key, file_segment.offset, file_segment.kind);
const auto path = cache->getFileSegmentPath(
file_segment.key, file_segment.offset, file_segment.kind,
FileCache::UserInfo(file_segment.user_id, file_segment.user_weight));
res_columns[i++]->insert(path);
res_columns[i++]->insert(file_segment.key.toString());
res_columns[i++]->insert(file_segment.range_left);
@ -67,6 +70,7 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex
res_columns[i++]->insert(file_segment.downloaded_size);
res_columns[i++]->insert(toString(file_segment.kind));
res_columns[i++]->insert(file_segment.is_unbound);
res_columns[i++]->insert(file_segment.user_id);
std::error_code ec;
auto size = fs::file_size(path, ec);
@ -74,7 +78,7 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex
res_columns[i++]->insert(size);
else
res_columns[i++]->insertDefault();
});
}, FileCache::getCommonUser().user_id);
}
}

View File

@ -373,11 +373,37 @@ QueryTreeNodePtr buildQueryTreeForShard(SelectQueryInfo & query_info, QueryTreeN
removeGroupingFunctionSpecializations(query_tree_to_modify);
// std::cerr << "====================== build 1 \n" << query_tree_to_modify->dumpTree() << std::endl;
createUniqueTableAliases(query_tree_to_modify, nullptr, planner_context->getQueryContext());
// std::cerr << "====================== build 2 \n" << query_tree_to_modify->dumpTree() << std::endl;
return query_tree_to_modify;
}
class RewriteJoinToGlobalJoinVisitor : public InDepthQueryTreeVisitor<RewriteJoinToGlobalJoinVisitor>
{
public:
using Base = InDepthQueryTreeVisitor<RewriteJoinToGlobalJoinVisitor>;
using Base::Base;
void visitImpl(QueryTreeNodePtr & node)
{
if (auto * join_node = node->as<JoinNode>())
join_node->setLocality(JoinLocality::Global);
}
static bool needChildVisit(QueryTreeNodePtr & parent, QueryTreeNodePtr & child)
{
auto * join_node = parent->as<JoinNode>();
if (join_node && join_node->getRightTableExpression() == child)
return false;
return true;
}
};
void rewriteJoinToGlobalJoin(QueryTreeNodePtr query_tree_to_modify)
{
RewriteJoinToGlobalJoinVisitor visitor;
visitor.visit(query_tree_to_modify);
}
}

View File

@ -12,4 +12,6 @@ using QueryTreeNodePtr = std::shared_ptr<IQueryTreeNode>;
QueryTreeNodePtr buildQueryTreeForShard(SelectQueryInfo & query_info, QueryTreeNodePtr query_tree_to_modify);
void rewriteJoinToGlobalJoin(QueryTreeNodePtr query_tree_to_modify);
}

View File

@ -1,7 +1,7 @@
#!/usr/bin/env python3
from pathlib import Path
from typing import List, Tuple
from typing import List, Tuple, Optional
import argparse
import csv
import logging
@ -32,7 +32,8 @@ def post_commit_status_from_file(file_path: Path) -> List[str]:
return res[0]
def process_result(file_path: Path) -> Tuple[bool, TestResults]:
# Returns (is_ok, test_results, error_message)
def process_result(file_path: Path) -> Tuple[bool, TestResults, Optional[str]]:
test_results = [] # type: TestResults
state, report_url, description = post_commit_status_from_file(file_path)
prefix = file_path.parent.name
@ -46,11 +47,11 @@ def process_result(file_path: Path) -> Tuple[bool, TestResults]:
if report_url != "null"
else "Check failed"
)
return False, [TestResult(f"{prefix}: {description}", status)]
return False, [TestResult(f"{prefix}: {description}", status)], "Check failed"
is_ok = state == "success"
if is_ok and report_url == "null":
return is_ok, test_results
return is_ok, test_results, None
status = (
f'OK: Bug reproduced (<a href="{report_url}">Report</a>)'
@ -58,19 +59,22 @@ def process_result(file_path: Path) -> Tuple[bool, TestResults]:
else f'Bug is not reproduced (<a href="{report_url}">Report</a>)'
)
test_results.append(TestResult(f"{prefix}: {description}", status))
return is_ok, test_results
return is_ok, test_results, None
def process_all_results(file_paths: List[Path]) -> Tuple[bool, TestResults]:
def process_all_results(
file_paths: List[Path],
) -> Tuple[bool, TestResults, Optional[str]]:
any_ok = False
all_results = []
error = None
for status_path in file_paths:
is_ok, test_results = process_result(status_path)
is_ok, test_results, error = process_result(status_path)
any_ok = any_ok or is_ok
if test_results is not None:
all_results.extend(test_results)
return any_ok, all_results
return any_ok and error is None, all_results, error
def main():
@ -80,7 +84,13 @@ def main():
check_name_with_group = "Bugfix validate check"
is_ok, test_results = process_all_results(status_files)
is_ok, test_results, error = process_all_results(status_files)
description = ""
if error:
description = error
elif not is_ok:
description = "Changed tests don't reproduce the bug"
pr_info = PRInfo()
if not test_results:
@ -88,7 +98,6 @@ def main():
report_url = ""
logging.info("No results to upload")
else:
description = "" if is_ok else "Changed tests don't reproduce the bug"
report_url = upload_results(
S3Helper(),
pr_info.number,

View File

@ -118,7 +118,12 @@ class Cache:
def _download(self, url: str, ignore_error: bool = False) -> None:
compressed_cache = self.temp_path / self.archive_name
try:
download_build_with_progress(url, compressed_cache)
if url.startswith("file://"):
local_s3_cache = Path(url[7:])
if local_s3_cache.is_file():
shutil.copy2(local_s3_cache, compressed_cache)
else:
download_build_with_progress(url, compressed_cache)
except DownloadException as e:
if not ignore_error:
raise CacheError(f"Failed to download {url}") from e

View File

@ -32,6 +32,7 @@ from pathlib import Path
from subprocess import CalledProcessError
from typing import List, Optional
import __main__
from env_helper import TEMP_PATH
from get_robot_token import get_best_robot_token
from git_helper import git_runner, is_shallow
@ -606,16 +607,18 @@ def parse_args():
@contextmanager
def clear_repo():
orig_ref = git_runner("git branch --show-current") or git_runner(
"git rev-parse HEAD"
)
def ref():
return git_runner("git branch --show-current") or git_runner(
"git rev-parse HEAD"
)
orig_ref = ref()
try:
yield
except (Exception, KeyboardInterrupt):
git_runner(f"git checkout -f {orig_ref}")
raise
else:
git_runner(f"git checkout -f {orig_ref}")
finally:
current_ref = ref()
if orig_ref != current_ref:
git_runner(f"git checkout -f {orig_ref}")
@contextmanager
@ -623,15 +626,14 @@ def stash():
# diff.ignoreSubmodules=all don't show changed submodules
need_stash = bool(git_runner("git -c diff.ignoreSubmodules=all diff HEAD"))
if need_stash:
git_runner("git stash push --no-keep-index -m 'running cherry_pick.py'")
script = (
__main__.__file__ if hasattr(__main__, "__file__") else "unknown script"
)
git_runner(f"git stash push --no-keep-index -m 'running {script}'")
try:
with clear_repo():
yield
except (Exception, KeyboardInterrupt):
if need_stash:
git_runner("git stash pop")
raise
else:
finally:
if need_stash:
git_runner("git stash pop")

View File

@ -82,6 +82,8 @@ class BuildConfig:
"./contrib/libmetrohash",
"./contrib/update-submodules.sh",
"./contrib/CMakeLists.txt",
"./CMakeLists.txt",
"./PreLoad.cmake",
"./cmake",
"./base",
"./programs",
@ -133,19 +135,32 @@ install_check_digest = DigestConfig(
include_paths=["./tests/ci/install_check.py"],
docker=["clickhouse/install-deb-test", "clickhouse/install-rpm-test"],
)
statless_check_digest = DigestConfig(
include_paths=["./tests/queries/0_stateless/"],
stateless_check_digest = DigestConfig(
include_paths=[
"./tests/queries/0_stateless/",
"./tests/clickhouse-test",
"./tests/*.txt",
],
exclude_files=[".md"],
docker=["clickhouse/stateless-test"],
)
stateful_check_digest = DigestConfig(
include_paths=["./tests/queries/1_stateful/"],
include_paths=[
"./tests/queries/1_stateful/",
"./tests/clickhouse-test",
"./tests/*.txt",
],
exclude_files=[".md"],
docker=["clickhouse/stateful-test"],
)
# FIXME: which tests are stresstest? stateless?
stress_check_digest = DigestConfig(
include_paths=["./tests/queries/0_stateless/"],
include_paths=[
"./tests/queries/0_stateless/",
"./tests/queries/1_stateful/",
"./tests/clickhouse-test",
"./tests/*.txt",
],
exclude_files=[".md"],
docker=["clickhouse/stress-test"],
)
@ -209,7 +224,7 @@ bugfix_validate_check = DigestConfig(
)
# common test params
statless_test_common_params = {
"digest": statless_check_digest,
"digest": stateless_check_digest,
"run_command": 'functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT',
"timeout": 10800,
}
@ -474,7 +489,7 @@ CI_CONFIG = CiConfig(
compiler="clang-17",
debug_build=True,
package_type="deb",
sparse_checkout=True,
sparse_checkout=True, # Check that it works with at least one build, see also update-submodules.sh
),
"binary_release": BuildConfig(
name="binary_release",
@ -495,7 +510,7 @@ CI_CONFIG = CiConfig(
compiler="clang-17-darwin",
package_type="binary",
static_binary_name="macos",
sparse_checkout=True,
sparse_checkout=True, # Check that it works with at least one build, see also update-submodules.sh
),
"binary_aarch64": BuildConfig(
name="binary_aarch64",
@ -825,10 +840,18 @@ CI_CONFIG = CiConfig(
"package_asan",
job_config=JobConfig(**{**statless_test_common_params, "timeout": 3600}), # type: ignore
),
# FIXME: add digest and params
"ClickHouse Keeper Jepsen": TestConfig("binary_release"),
# FIXME: add digest and params
"ClickHouse Server Jepsen": TestConfig("binary_release"),
"ClickHouse Keeper Jepsen": TestConfig(
"binary_release",
job_config=JobConfig(
run_by_label="jepsen-test", run_command="jepsen_check.py keeper"
),
),
"ClickHouse Server Jepsen": TestConfig(
"binary_release",
job_config=JobConfig(
run_by_label="jepsen-test", run_command="jepsen_check.py server"
),
),
"Performance Comparison": TestConfig(
"package_release",
job_config=JobConfig(num_batches=4, **perf_test_common_params), # type: ignore

View File

@ -92,7 +92,7 @@ class Runner:
return
self._cwd = value
def __call__(self, *args, **kwargs):
def __call__(self, *args: Any, **kwargs: Any) -> str:
return self.run(*args, **kwargs)

View File

@ -10,14 +10,18 @@ from pathlib import Path
from typing import Any, List
import boto3 # type: ignore
import requests # type: ignore
from github import Github
from build_download_helper import get_build_name_for_check
import requests # type: ignore
from git_helper import git_runner
from build_download_helper import (
download_build_with_progress,
get_build_name_for_check,
read_build_urls,
)
from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse
from commit_status_helper import RerunHelper, get_commit, post_commit_status
from compress_files import compress_fast
from env_helper import REPO_COPY, TEMP_PATH, S3_BUILDS_BUCKET, S3_DOWNLOAD
from env_helper import REPO_COPY, REPORT_PATH, S3_BUILDS_BUCKET, S3_URL, TEMP_PATH
from get_robot_token import get_best_robot_token, get_parameter_from_ssm
from pr_info import PRInfo
from report import TestResults, TestResult
@ -26,8 +30,6 @@ from ssh import SSHKey
from stopwatch import Stopwatch
from tee_popen import TeePopen
from upload_result_helper import upload_results
from version_helper import get_version_from_repo
from build_check import get_release_or_pr
JEPSEN_GROUP_NAME = "jepsen_group"
@ -212,27 +214,36 @@ def main():
# always use latest
docker_image = KEEPER_IMAGE_NAME if args.program == "keeper" else SERVER_IMAGE_NAME
build_name = get_build_name_for_check(check_name)
release_or_pr, _ = get_release_or_pr(pr_info, get_version_from_repo())
# This check run separately from other checks because it requires exclusive
# run (see .github/workflows/jepsen.yml) So we cannot add explicit
# dependency on a build job and using busy loop on it's results. For the
# same reason we are using latest docker image.
build_url = (
f"{S3_DOWNLOAD}/{S3_BUILDS_BUCKET}/{release_or_pr}/{pr_info.sha}/"
f"{build_name}/clickhouse"
)
head = requests.head(build_url)
counter = 0
while head.status_code != 200:
time.sleep(10)
if pr_info.is_scheduled() or pr_info.is_dispatched():
# get latest clcikhouse by the static link for latest master buit - get its version and provide permanent url for this version to the jepsen
build_url = f"{S3_URL}/{S3_BUILDS_BUCKET}/master/amd64/clickhouse"
download_build_with_progress(build_url, Path(TEMP_PATH) / "clickhouse")
git_runner.run(f"chmod +x {TEMP_PATH}/clickhouse")
sha = git_runner.run(
f"{TEMP_PATH}/clickhouse local -q \"select value from system.build_options where name='GIT_HASH'\""
)
version_full = git_runner.run(
f'{TEMP_PATH}/clickhouse local -q "select version()"'
)
version = ".".join(version_full.split(".")[0:2])
assert len(sha) == 40, f"failed to fetch sha from the binary. result: {sha}"
assert (
version
), f"failed to fetch version from the binary. result: {version_full}"
build_url = (
f"{S3_URL}/{S3_BUILDS_BUCKET}/{version}/{sha}/binary_release/clickhouse"
)
print(f"Clickhouse version: [{version_full}], sha: [{sha}], url: [{build_url}]")
head = requests.head(build_url)
counter += 1
if counter >= 180:
logging.warning("Cannot fetch build in 30 minutes, exiting")
sys.exit(0)
assert head.status_code == 200, f"Clickhouse binary not found: {build_url}"
else:
build_name = get_build_name_for_check(check_name)
urls = read_build_urls(build_name, REPORT_PATH)
build_url = None
for url in urls:
if url.endswith("clickhouse"):
build_url = url
assert build_url, "No build url found in the report"
extra_args = ""
if args.program == "server":

View File

@ -38,6 +38,14 @@ DIFF_IN_DOCUMENTATION_EXT = [
RETRY_SLEEP = 0
class EventType:
UNKNOWN = 0
PUSH = 1
PULL_REQUEST = 2
SCHEDULE = 3
DISPATCH = 4
def get_pr_for_commit(sha, ref):
if not ref:
return None
@ -99,6 +107,7 @@ class PRInfo:
# release_pr and merged_pr are used for docker images additional cache
self.release_pr = 0
self.merged_pr = 0
self.event_type = EventType.UNKNOWN
ref = github_event.get("ref", "refs/heads/master")
if ref and ref.startswith("refs/heads/"):
ref = ref[11:]
@ -115,6 +124,7 @@ class PRInfo:
github_event["pull_request"] = prs_for_sha[0]
if "pull_request" in github_event: # pull request and other similar events
self.event_type = EventType.PULL_REQUEST
self.number = github_event["pull_request"]["number"] # type: int
if pr_event_from_api:
try:
@ -175,6 +185,7 @@ class PRInfo:
self.diff_urls.append(self.compare_pr_url(github_event["pull_request"]))
elif "commits" in github_event:
self.event_type = EventType.PUSH
# `head_commit` always comes with `commits`
commit_message = github_event["head_commit"]["message"] # type: str
if commit_message.startswith("Merge pull request #"):
@ -243,6 +254,11 @@ class PRInfo:
)
)
else:
if "schedule" in github_event:
self.event_type = EventType.SCHEDULE
else:
# assume this is a dispatch
self.event_type = EventType.DISPATCH
print("event.json does not match pull_request or push:")
print(json.dumps(github_event, sort_keys=True, indent=4))
self.sha = os.getenv(
@ -263,6 +279,12 @@ class PRInfo:
if need_changed_files:
self.fetch_changed_files()
def is_scheduled(self):
return self.event_type == EventType.SCHEDULE
def is_dispatched(self):
return self.event_type == EventType.DISPATCH
def compare_pr_url(self, pr_object: dict) -> str:
return self.compare_url(pr_object["base"]["label"], pr_object["head"]["label"])

View File

@ -744,7 +744,7 @@ def create_build_html_report(
build_result.build_config is not None
and build_result.build_config.sparse_checkout
):
comment += " (note: sparse checkout is used)"
comment += " (note: sparse checkout is used, see update-submodules.sh)"
row.append(f"<td>{comment}</td>")
row.append("</tr>")

View File

@ -948,6 +948,8 @@ class TestCase:
else ""
)
self.show_whitespaces_in_diff = args.show_whitespaces_in_diff
# should skip test, should increment skipped_total, skip reason
def should_skip_test(self, suite) -> Optional[FailureReason]:
tags = self.tags
@ -1171,7 +1173,7 @@ class TestCase:
)
if result_is_different:
diff = Popen(
diff_proc = Popen(
[
"diff",
"-U",
@ -1182,7 +1184,18 @@ class TestCase:
encoding="latin-1",
stdout=PIPE,
universal_newlines=True,
).communicate()[0]
)
if self.show_whitespaces_in_diff:
sed_proc = Popen(
["sed", "-e", "s/[ \t]\\+$/&$/g"],
stdin=diff_proc.stdout,
stdout=PIPE,
)
diff_proc.stdout.close() # Allow diff to receive a SIGPIPE if cat exits.
diff = sed_proc.communicate()[0].decode("utf-8", errors="ignore")
else:
diff = diff_proc.communicate()[0]
if diff.startswith("Binary files "):
diff += "Content of stdout:\n===================\n"
file = open(self.stdout_file, "rb")
@ -2780,6 +2793,12 @@ def parse_args():
help="Total test groups for crc32(test_name) % run_by_hash_total == run_by_hash_num",
)
parser.add_argument(
"--show-whitespaces-in-diff",
action="store_true",
help="Display $ characters after line with trailing whitespaces in diff output",
)
group = parser.add_mutually_exclusive_group(required=False)
group.add_argument(
"--zookeeper",

Some files were not shown because too many files have changed in this diff Show More