changes around buffer sizes

This commit is contained in:
Nikita Taranov 2023-06-12 16:18:47 +02:00
parent e88fc39895
commit 1d33043fe6
3 changed files with 31 additions and 20 deletions

View File

@ -33,15 +33,6 @@ namespace ProfileEvents
extern const Event RemoteFSBuffers; extern const Event RemoteFSBuffers;
} }
namespace
{
size_t chooseBufferSizeForRemoteReading(const DB::ReadSettings & settings, size_t file_size)
{
/// Buffers used for prefetch or pre-download better to have enough size, but not bigger than the whole file.
return std::min<size_t>(std::max<size_t>(settings.prefetch_buffer_size, DBMS_DEFAULT_BUFFER_SIZE), file_size);
}
}
namespace DB namespace DB
{ {

View File

@ -2,14 +2,27 @@
#include <IO/SeekableReadBuffer.h> #include <IO/SeekableReadBuffer.h>
#include <iostream>
#include <Disks/IO/CachedOnDiskReadBufferFromFile.h> #include <Disks/IO/CachedOnDiskReadBufferFromFile.h>
#include <Disks/ObjectStorages/Cached/CachedObjectStorage.h> #include <Disks/ObjectStorages/Cached/CachedObjectStorage.h>
#include <Common/logger_useful.h> #include <IO/ReadSettings.h>
#include <IO/SwapHelper.h> #include <IO/SwapHelper.h>
#include <iostream>
#include <base/hex.h>
#include <Interpreters/FilesystemCacheLog.h> #include <Interpreters/FilesystemCacheLog.h>
#include <base/hex.h>
#include <Common/logger_useful.h>
using namespace DB;
namespace
{
bool withCache(const ReadSettings & settings)
{
return settings.remote_fs_cache && settings.enable_filesystem_cache
&& (!CurrentThread::getQueryId().empty() || settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache
|| !settings.avoid_readthrough_cache_outside_query_context);
}
}
namespace DB namespace DB
{ {
@ -18,29 +31,35 @@ namespace ErrorCodes
extern const int CANNOT_SEEK_THROUGH_FILE; extern const int CANNOT_SEEK_THROUGH_FILE;
} }
size_t chooseBufferSizeForRemoteReading(const DB::ReadSettings & settings, size_t file_size)
{
/// Only when cache is used we could download bigger portions of FileSegments than what we actually gonna read within particular task.
if (!withCache(settings))
return settings.remote_fs_buffer_size;
/// Buffers used for prefetch and pre-download better to have enough size, but not bigger than the whole file.
return std::min<size_t>(std::max<size_t>(settings.remote_fs_buffer_size, DBMS_DEFAULT_BUFFER_SIZE), file_size);
}
ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather( ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather(
ReadBufferCreator && read_buffer_creator_, ReadBufferCreator && read_buffer_creator_,
const StoredObjects & blobs_to_read_, const StoredObjects & blobs_to_read_,
const ReadSettings & settings_, const ReadSettings & settings_,
std::shared_ptr<FilesystemCacheLog> cache_log_, std::shared_ptr<FilesystemCacheLog> cache_log_,
bool use_external_buffer_) bool use_external_buffer_)
: ReadBufferFromFileBase(use_external_buffer_ ? 0 : settings_.remote_fs_buffer_size, nullptr, 0) : ReadBufferFromFileBase(
use_external_buffer_ ? 0 : chooseBufferSizeForRemoteReading(settings_, getTotalSize(blobs_to_read_)), nullptr, 0)
, settings(settings_) , settings(settings_)
, blobs_to_read(blobs_to_read_) , blobs_to_read(blobs_to_read_)
, read_buffer_creator(std::move(read_buffer_creator_)) , read_buffer_creator(std::move(read_buffer_creator_))
, cache_log(settings.enable_filesystem_cache_log ? cache_log_ : nullptr) , cache_log(settings.enable_filesystem_cache_log ? cache_log_ : nullptr)
, query_id(CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr ? CurrentThread::getQueryId() : "") , query_id(CurrentThread::getQueryId())
, use_external_buffer(use_external_buffer_) , use_external_buffer(use_external_buffer_)
, with_cache(withCache(settings))
, log(&Poco::Logger::get("ReadBufferFromRemoteFSGather")) , log(&Poco::Logger::get("ReadBufferFromRemoteFSGather"))
{ {
if (!blobs_to_read.empty()) if (!blobs_to_read.empty())
current_object = blobs_to_read.front(); current_object = blobs_to_read.front();
with_cache = settings.remote_fs_cache
&& settings.enable_filesystem_cache
&& (!query_id.empty()
|| settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache
|| !settings.avoid_readthrough_cache_outside_query_context);
} }
SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(const StoredObject & object) SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(const StoredObject & object)

View File

@ -86,4 +86,5 @@ private:
Poco::Logger * log; Poco::Logger * log;
}; };
size_t chooseBufferSizeForRemoteReading(const DB::ReadSettings & settings, size_t file_size);
} }