mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge remote-tracking branch 'blessed/master' into argmin_optimization
This commit is contained in:
commit
3739d46817
1
.github/workflows/jepsen.yml
vendored
1
.github/workflows/jepsen.yml
vendored
@ -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
|
||||
|
17
.github/workflows/pull_request.yml
vendored
17
.github/workflows/pull_request.yml
vendored
@ -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 ###########################################
|
||||
#############################################################################################
|
||||
|
1
.github/workflows/reusable_build.yml
vendored
1
.github/workflows/reusable_build.yml
vendored
@ -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'
|
||||
|
2
.github/workflows/reusable_simple_job.yml
vendored
2
.github/workflows/reusable_simple_job.yml
vendored
@ -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:
|
||||
|
6
contrib/update-submodules.sh
vendored
6
contrib/update-submodules.sh
vendored
@ -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.
|
||||
#
|
||||
|
@ -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
|
||||
|
@ -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:
|
||||
|
@ -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();
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include "AutoFinalOnQueryPass.h"
|
||||
#include <Analyzer/Passes/AutoFinalOnQueryPass.h>
|
||||
|
||||
#include <Storages/IStorage.h>
|
||||
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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,
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include "UniqToCountPass.h"
|
||||
#include <Analyzer/Passes/UniqToCountPass.h>
|
||||
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
@ -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_)
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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 = {});
|
||||
|
@ -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) \
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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.
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include "Core/DecimalFunctions.h"
|
||||
#include "DataTypes/IDataType.h"
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
};
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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,
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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)
|
||||
|
@ -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);
|
||||
|
@ -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));
|
||||
}
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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>
|
||||
|
||||
|
||||
|
@ -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));
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -173,4 +173,10 @@ void FileCacheFactory::updateSettingsFromConfig(const Poco::Util::AbstractConfig
|
||||
}
|
||||
}
|
||||
|
||||
void FileCacheFactory::clear()
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
caches_by_name.clear();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -55,6 +55,8 @@ public:
|
||||
|
||||
void updateSettingsFromConfig(const Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
void clear();
|
||||
|
||||
private:
|
||||
std::mutex mutex;
|
||||
CacheByName caches_by_name;
|
||||
|
@ -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);
|
||||
};
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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;
|
||||
|
@ -78,5 +78,7 @@ namespace DB
|
||||
uint64_t references;
|
||||
bool is_unbound;
|
||||
FileCacheQueueEntryType queue_entry_type;
|
||||
std::string user_id;
|
||||
uint64_t user_weight;
|
||||
};
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
@ -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())
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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());
|
||||
|
||||
|
@ -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,
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
24
src/Interpreters/Cache/UserInfo.h
Normal file
24
src/Interpreters/Cache/UserInfo.h
Normal 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; }
|
||||
};
|
||||
|
||||
}
|
@ -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()
|
||||
|
@ -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),
|
||||
});
|
||||
};
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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),
|
||||
|
@ -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,
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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");
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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(
|
||||
|
@ -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) });
|
||||
|
@ -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)
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -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_)
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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();
|
||||
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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);
|
||||
};
|
||||
}
|
||||
}
|
||||
|
@ -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,
|
||||
|
@ -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,
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
}
|
||||
|
@ -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,
|
||||
|
@ -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
|
||||
|
@ -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")
|
||||
|
||||
|
@ -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
|
||||
|
@ -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)
|
||||
|
||||
|
||||
|
@ -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":
|
||||
|
@ -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"])
|
||||
|
||||
|
@ -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>")
|
||||
|
@ -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
Loading…
Reference in New Issue
Block a user