Merge branch 'master' into kvm-clock

This commit is contained in:
Sergei Trifonov 2022-09-13 12:45:24 +02:00 committed by GitHub
commit 5ee2f1701c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
47 changed files with 982 additions and 598 deletions

View File

@ -38,6 +38,7 @@ FORMAT_SCHEMA_PATH="$(clickhouse extract-from-config --config-file "$CLICKHOUSE_
# There could be many disks declared in config # There could be many disks declared in config
readarray -t DISKS_PATHS < <(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key='storage_configuration.disks.*.path' || true) readarray -t DISKS_PATHS < <(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key='storage_configuration.disks.*.path' || true)
readarray -t DISKS_METADATA_PATHS < <(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key='storage_configuration.disks.*.metadata_path' || true)
CLICKHOUSE_USER="${CLICKHOUSE_USER:-default}" CLICKHOUSE_USER="${CLICKHOUSE_USER:-default}"
CLICKHOUSE_PASSWORD="${CLICKHOUSE_PASSWORD:-}" CLICKHOUSE_PASSWORD="${CLICKHOUSE_PASSWORD:-}"
@ -50,7 +51,8 @@ for dir in "$DATA_DIR" \
"$TMP_DIR" \ "$TMP_DIR" \
"$USER_PATH" \ "$USER_PATH" \
"$FORMAT_SCHEMA_PATH" \ "$FORMAT_SCHEMA_PATH" \
"${DISKS_PATHS[@]}" "${DISKS_PATHS[@]}" \
"${DISKS_METADATA_PATHS[@]}"
do do
# check if variable not empty # check if variable not empty
[ -z "$dir" ] && continue [ -z "$dir" ] && continue

View File

@ -370,6 +370,7 @@ else
# Avoid "Setting s3_check_objects_after_upload is neither a builtin setting..." # Avoid "Setting s3_check_objects_after_upload is neither a builtin setting..."
rm -f /etc/clickhouse-server/users.d/enable_blobs_check.xml ||: rm -f /etc/clickhouse-server/users.d/enable_blobs_check.xml ||:
rm -f /etc/clickhouse-server/users.d/marks.xml ||:
# Remove s3 related configs to avoid "there is no disk type `cache`" # Remove s3 related configs to avoid "there is no disk type `cache`"
rm -f /etc/clickhouse-server/config.d/storage_conf.xml ||: rm -f /etc/clickhouse-server/config.d/storage_conf.xml ||:

View File

@ -227,6 +227,8 @@ void LocalServer::cleanup()
global_context.reset(); global_context.reset();
} }
/// thread status should be destructed before shared context because it relies on process list.
status.reset(); status.reset();
// Delete the temporary directory if needed. // Delete the temporary directory if needed.
@ -366,7 +368,7 @@ int LocalServer::main(const std::vector<std::string> & /*args*/)
try try
{ {
UseSSL use_ssl; UseSSL use_ssl;
ThreadStatus thread_status; thread_status.emplace();
StackTrace::setShowAddresses(config().getBool("show_addresses_in_stack_traces", true)); StackTrace::setShowAddresses(config().getBool("show_addresses_in_stack_traces", true));

View File

@ -176,9 +176,6 @@ protected:
bool stderr_is_a_tty = false; /// stderr is a terminal. bool stderr_is_a_tty = false; /// stderr is a terminal.
uint64_t terminal_width = 0; uint64_t terminal_width = 0;
ServerConnectionPtr connection;
ConnectionParameters connection_parameters;
String format; /// Query results output format. String format; /// Query results output format.
bool select_into_file = false; /// If writing result INTO OUTFILE. It affects progress rendering. bool select_into_file = false; /// If writing result INTO OUTFILE. It affects progress rendering.
bool select_into_file_and_stdout = false; /// If writing result INTO OUTFILE AND STDOUT. It affects progress rendering. bool select_into_file_and_stdout = false; /// If writing result INTO OUTFILE AND STDOUT. It affects progress rendering.
@ -199,6 +196,11 @@ protected:
SharedContextHolder shared_context; SharedContextHolder shared_context;
ContextMutablePtr global_context; ContextMutablePtr global_context;
std::optional<ThreadStatus> thread_status;
ServerConnectionPtr connection;
ConnectionParameters connection_parameters;
/// Buffer that reads from stdin in batch mode. /// Buffer that reads from stdin in batch mode.
ReadBufferFromFileDescriptor std_in{STDIN_FILENO}; ReadBufferFromFileDescriptor std_in{STDIN_FILENO};
/// Console output. /// Console output.

View File

@ -31,9 +31,6 @@ LocalConnection::LocalConnection(ContextPtr context_, bool send_progress_, bool
/// Authenticate and create a context to execute queries. /// Authenticate and create a context to execute queries.
session.authenticate("default", "", Poco::Net::SocketAddress{}); session.authenticate("default", "", Poco::Net::SocketAddress{});
session.makeSessionContext(); session.makeSessionContext();
if (!CurrentThread::isInitialized())
thread_status.emplace();
} }
LocalConnection::~LocalConnection() LocalConnection::~LocalConnection()

View File

@ -156,7 +156,6 @@ private:
String description = "clickhouse-local"; String description = "clickhouse-local";
std::optional<LocalQueryState> state; std::optional<LocalQueryState> state;
std::optional<ThreadStatus> thread_status;
/// Last "server" packet. /// Last "server" packet.
std::optional<UInt64> next_packet_type; std::optional<UInt64> next_packet_type;

View File

@ -146,6 +146,9 @@
M(SelectedRows, "Number of rows SELECTed from all tables.") \ M(SelectedRows, "Number of rows SELECTed from all tables.") \
M(SelectedBytes, "Number of bytes (uncompressed; for columns as they stored in memory) SELECTed from all tables.") \ M(SelectedBytes, "Number of bytes (uncompressed; for columns as they stored in memory) SELECTed from all tables.") \
\ \
M(WaitMarksLoadMicroseconds, "Time spent loading marks") \
M(BackgroundLoadingMarksTasks, "Number of background tasks for loading marks") \
\
M(Merge, "Number of launched background merges.") \ M(Merge, "Number of launched background merges.") \
M(MergedRows, "Rows read for background merges. This is the number of rows before merge.") \ M(MergedRows, "Rows read for background merges. This is the number of rows before merge.") \
M(MergedUncompressedBytes, "Uncompressed bytes (for columns as they stored in memory) that was read for background merges. This is the number before merge.") \ M(MergedUncompressedBytes, "Uncompressed bytes (for columns as they stored in memory) that was read for background merges. This is the number before merge.") \

View File

@ -442,9 +442,9 @@ void KeeperServer::shutdownRaftServer()
void KeeperServer::shutdown() void KeeperServer::shutdown()
{ {
state_machine->shutdownStorage();
state_manager->flushAndShutDownLogStore(); state_manager->flushAndShutDownLogStore();
shutdownRaftServer(); shutdownRaftServer();
state_machine->shutdownStorage();
} }
namespace namespace
@ -617,7 +617,9 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ
auto & entry_buf = entry->get_buf(); auto & entry_buf = entry->get_buf();
auto request_for_session = state_machine->parseRequest(entry_buf); auto request_for_session = state_machine->parseRequest(entry_buf);
request_for_session.zxid = next_zxid; request_for_session.zxid = next_zxid;
state_machine->preprocess(request_for_session); if (!state_machine->preprocess(request_for_session))
return nuraft::cb_func::ReturnCode::ReturnNull;
request_for_session.digest = state_machine->getNodesDigest(); request_for_session.digest = state_machine->getNodesDigest();
entry = nuraft::cs_new<nuraft::log_entry>(entry->get_term(), getZooKeeperLogEntry(request_for_session), entry->get_val_type()); entry = nuraft::cs_new<nuraft::log_entry>(entry->get_term(), getZooKeeperLogEntry(request_for_session), entry->get_val_type());
break; break;

View File

@ -191,12 +191,16 @@ KeeperStorage::RequestForSession KeeperStateMachine::parseRequest(nuraft::buffer
return request_for_session; return request_for_session;
} }
void KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & request_for_session) bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & request_for_session)
{ {
if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID)
return; return true;
std::lock_guard lock(storage_and_responses_lock); std::lock_guard lock(storage_and_responses_lock);
if (storage->isFinalized())
return false;
try try
{ {
storage->preprocessRequest( storage->preprocessRequest(
@ -215,6 +219,8 @@ void KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req
if (keeper_context->digest_enabled && request_for_session.digest) if (keeper_context->digest_enabled && request_for_session.digest)
assertDigest(*request_for_session.digest, storage->getNodesDigest(false), *request_for_session.request, false); assertDigest(*request_for_session.digest, storage->getNodesDigest(false), *request_for_session.request, false);
return true;
} }
nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data) nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data)

View File

@ -33,7 +33,7 @@ public:
static KeeperStorage::RequestForSession parseRequest(nuraft::buffer & data); static KeeperStorage::RequestForSession parseRequest(nuraft::buffer & data);
void preprocess(const KeeperStorage::RequestForSession & request_for_session); bool preprocess(const KeeperStorage::RequestForSession & request_for_session);
nuraft::ptr<nuraft::buffer> pre_commit(uint64_t log_idx, nuraft::buffer & data) override; nuraft::ptr<nuraft::buffer> pre_commit(uint64_t log_idx, nuraft::buffer & data) override;

View File

@ -516,7 +516,7 @@ void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid)
std::shared_ptr<KeeperStorage::Node> KeeperStorage::UncommittedState::getNode(StringRef path) const std::shared_ptr<KeeperStorage::Node> KeeperStorage::UncommittedState::getNode(StringRef path) const
{ {
if (auto node_it = nodes.find(std::string{path}); node_it != nodes.end()) if (auto node_it = nodes.find(path.toView()); node_it != nodes.end())
return node_it->second.node; return node_it->second.node;
return tryGetNodeFromStorage(path); return tryGetNodeFromStorage(path);
@ -524,7 +524,7 @@ std::shared_ptr<KeeperStorage::Node> KeeperStorage::UncommittedState::getNode(St
Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef path) const Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef path) const
{ {
if (auto node_it = nodes.find(std::string{path}); node_it != nodes.end()) if (auto node_it = nodes.find(path.toView()); node_it != nodes.end())
return node_it->second.acls; return node_it->second.acls;
auto node_it = storage.container.find(path); auto node_it = storage.container.find(path);
@ -830,7 +830,9 @@ bool KeeperStorage::checkACL(StringRef path, int32_t permission, int64_t session
void KeeperStorage::unregisterEphemeralPath(int64_t session_id, const std::string & path) void KeeperStorage::unregisterEphemeralPath(int64_t session_id, const std::string & path)
{ {
auto ephemerals_it = ephemerals.find(session_id); auto ephemerals_it = ephemerals.find(session_id);
assert(ephemerals_it != ephemerals.end()); if (ephemerals_it == ephemerals.end())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Session {} is missing ephemeral path");
ephemerals_it->second.erase(path); ephemerals_it->second.erase(path);
if (ephemerals_it->second.empty()) if (ephemerals_it->second.empty())
ephemerals.erase(ephemerals_it); ephemerals.erase(ephemerals_it);
@ -1825,6 +1827,11 @@ void KeeperStorage::finalize()
session_expiry_queue.clear(); session_expiry_queue.clear();
} }
bool KeeperStorage::isFinalized() const
{
return finalized;
}
class KeeperStorageRequestProcessorsFactory final : private boost::noncopyable class KeeperStorageRequestProcessorsFactory final : private boost::noncopyable
{ {
@ -1892,7 +1899,7 @@ UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vec
if (!keeper_context->digest_enabled) if (!keeper_context->digest_enabled)
return current_digest; return current_digest;
std::unordered_map<std::string, std::shared_ptr<Node>> updated_nodes; std::unordered_map<std::string_view, std::shared_ptr<Node>> updated_nodes;
for (const auto & delta : new_deltas) for (const auto & delta : new_deltas)
{ {

View File

@ -425,6 +425,8 @@ public:
void finalize(); void finalize();
bool isFinalized() const;
/// Set of methods for creating snapshots /// Set of methods for creating snapshots
/// Turn on snapshot mode, so data inside Container is not deleted, but replaced with new version. /// Turn on snapshot mode, so data inside Container is not deleted, but replaced with new version.

View File

@ -602,6 +602,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
M(Bool, skip_download_if_exceeds_query_cache, true, "Skip download from remote filesystem if exceeds query cache size", 0) \ M(Bool, skip_download_if_exceeds_query_cache, true, "Skip download from remote filesystem if exceeds query cache size", 0) \
M(UInt64, max_query_cache_size, (128UL * 1024 * 1024 * 1024), "Max remote filesystem cache size that can be used by a single query", 0) \ M(UInt64, max_query_cache_size, (128UL * 1024 * 1024 * 1024), "Max remote filesystem cache size that can be used by a single query", 0) \
\ \
M(Bool, load_marks_asynchronously, false, "Load MergeTree marks asynchronously", 0) \
\
M(Bool, use_structure_from_insertion_table_in_table_functions, false, "Use structure from insertion table instead of schema inference from data", 0) \ M(Bool, use_structure_from_insertion_table_in_table_functions, false, "Use structure from insertion table instead of schema inference from data", 0) \
\ \
M(UInt64, http_max_tries, 10, "Max attempts to read via http.", 0) \ M(UInt64, http_max_tries, 10, "Max attempts to read via http.", 0) \

View File

@ -118,7 +118,11 @@ void CachedOnDiskReadBufferFromFile::initialize(size_t offset, size_t size)
} }
else else
{ {
file_segments_holder.emplace(cache->getOrSet(cache_key, offset, size, is_persistent)); CreateFileSegmentSettings create_settings{
.is_persistent = is_persistent
};
file_segments_holder.emplace(cache->getOrSet(cache_key, offset, size, create_settings));
} }
/** /**
@ -156,7 +160,7 @@ CachedOnDiskReadBufferFromFile::getCacheReadBuffer(size_t offset) const
} }
CachedOnDiskReadBufferFromFile::ImplementationBufferPtr CachedOnDiskReadBufferFromFile::ImplementationBufferPtr
CachedOnDiskReadBufferFromFile::getRemoteFSReadBuffer(FileSegmentPtr & file_segment, ReadType read_type_) CachedOnDiskReadBufferFromFile::getRemoteFSReadBuffer(FileSegment & file_segment, ReadType read_type_)
{ {
switch (read_type_) switch (read_type_)
{ {
@ -178,7 +182,7 @@ CachedOnDiskReadBufferFromFile::getRemoteFSReadBuffer(FileSegmentPtr & file_segm
* Implementation buffer from segment1 is passed to segment2 once segment1 is loaded. * Implementation buffer from segment1 is passed to segment2 once segment1 is loaded.
*/ */
auto remote_fs_segment_reader = file_segment->getRemoteFileReader(); auto remote_fs_segment_reader = file_segment.getRemoteFileReader();
if (!remote_fs_segment_reader) if (!remote_fs_segment_reader)
{ {
@ -189,7 +193,7 @@ CachedOnDiskReadBufferFromFile::getRemoteFSReadBuffer(FileSegmentPtr & file_segm
ErrorCodes::CANNOT_USE_CACHE, ErrorCodes::CANNOT_USE_CACHE,
"Cache cannot be used with a ReadBuffer which does not support right bounded reads"); "Cache cannot be used with a ReadBuffer which does not support right bounded reads");
file_segment->setRemoteFileReader(remote_fs_segment_reader); file_segment.setRemoteFileReader(remote_fs_segment_reader);
} }
return remote_fs_segment_reader; return remote_fs_segment_reader;
@ -201,8 +205,8 @@ CachedOnDiskReadBufferFromFile::getRemoteFSReadBuffer(FileSegmentPtr & file_segm
if (remote_file_reader && remote_file_reader->getFileOffsetOfBufferEnd() == file_offset_of_buffer_end) if (remote_file_reader && remote_file_reader->getFileOffsetOfBufferEnd() == file_offset_of_buffer_end)
return remote_file_reader; return remote_file_reader;
auto remote_fs_segment_reader = file_segment->extractRemoteFileReader(); auto remote_fs_segment_reader = file_segment.extractRemoteFileReader();
if (remote_fs_segment_reader) if (remote_fs_segment_reader && file_offset_of_buffer_end == remote_file_reader->getFileOffsetOfBufferEnd())
remote_file_reader = remote_fs_segment_reader; remote_file_reader = remote_fs_segment_reader;
else else
remote_file_reader = implementation_buffer_creator(); remote_file_reader = implementation_buffer_creator();
@ -217,6 +221,19 @@ CachedOnDiskReadBufferFromFile::getRemoteFSReadBuffer(FileSegmentPtr & file_segm
} }
} }
bool CachedOnDiskReadBufferFromFile::canStartFromCache(size_t current_offset, const FileSegment & file_segment)
{
/// segment{k} state: DOWNLOADING
/// cache: [______|___________
/// ^
/// first_non_downloaded_offset (in progress)
/// requested_range: [__________]
/// ^
/// current_offset
size_t first_non_downloaded_offset = file_segment.getFirstNonDownloadedOffset();
return first_non_downloaded_offset > current_offset;
}
CachedOnDiskReadBufferFromFile::ImplementationBufferPtr CachedOnDiskReadBufferFromFile::ImplementationBufferPtr
CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & file_segment) CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & file_segment)
{ {
@ -236,7 +253,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
{ {
LOG_DEBUG(log, "Bypassing cache because `read_from_filesystem_cache_if_exists_otherwise_bypass_cache` option is used"); LOG_DEBUG(log, "Bypassing cache because `read_from_filesystem_cache_if_exists_otherwise_bypass_cache` option is used");
read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE; read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE;
return getRemoteFSReadBuffer(file_segment, read_type); return getRemoteFSReadBuffer(*file_segment, read_type);
} }
} }
@ -248,21 +265,16 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
{ {
LOG_DEBUG(log, "Bypassing cache because file segment state is `SKIP_CACHE`"); LOG_DEBUG(log, "Bypassing cache because file segment state is `SKIP_CACHE`");
read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE; read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE;
return getRemoteFSReadBuffer(file_segment, read_type); return getRemoteFSReadBuffer(*file_segment, read_type);
} }
case FileSegment::State::DOWNLOADING: case FileSegment::State::DOWNLOADING:
{ {
size_t download_offset = file_segment->getDownloadOffset(); if (canStartFromCache(file_offset_of_buffer_end, *file_segment))
bool can_start_from_cache = download_offset > file_offset_of_buffer_end;
/// If file segment is being downloaded but we can already read
/// from already downloaded part, do that.
if (can_start_from_cache)
{ {
/// segment{k} state: DOWNLOADING /// segment{k} state: DOWNLOADING
/// cache: [______|___________ /// cache: [______|___________
/// ^ /// ^
/// download_offset (in progress) /// first_non_downloaded_offset (in progress)
/// requested_range: [__________] /// requested_range: [__________]
/// ^ /// ^
/// file_offset_of_buffer_end /// file_offset_of_buffer_end
@ -282,12 +294,12 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
case FileSegment::State::EMPTY: case FileSegment::State::EMPTY:
case FileSegment::State::PARTIALLY_DOWNLOADED: case FileSegment::State::PARTIALLY_DOWNLOADED:
{ {
if (file_segment->getDownloadOffset() > file_offset_of_buffer_end) if (canStartFromCache(file_offset_of_buffer_end, *file_segment))
{ {
/// segment{k} state: PARTIALLY_DOWNLOADED /// segment{k} state: PARTIALLY_DOWNLOADED
/// cache: [______|___________ /// cache: [______|___________
/// ^ /// ^
/// download_offset (in progress) /// first_non_downloaded_offset (in progress)
/// requested_range: [__________] /// requested_range: [__________]
/// ^ /// ^
/// file_offset_of_buffer_end /// file_offset_of_buffer_end
@ -299,20 +311,12 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
auto downloader_id = file_segment->getOrSetDownloader(); auto downloader_id = file_segment->getOrSetDownloader();
if (downloader_id == file_segment->getCallerId()) if (downloader_id == file_segment->getCallerId())
{ {
size_t download_offset = file_segment->getDownloadOffset(); if (canStartFromCache(file_offset_of_buffer_end, *file_segment))
bool can_start_from_cache = download_offset > file_offset_of_buffer_end;
LOG_TEST(
log,
"Current download offset: {}, file offset of buffer end: {}",
download_offset, file_offset_of_buffer_end);
if (can_start_from_cache)
{ {
/// segment{k} /// segment{k}
/// cache: [______|___________ /// cache: [______|___________
/// ^ /// ^
/// download_offset /// first_non_downloaded_offset
/// requested_range: [__________] /// requested_range: [__________]
/// ^ /// ^
/// file_offset_of_buffer_end /// file_offset_of_buffer_end
@ -322,27 +326,24 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
return getCacheReadBuffer(range.left); return getCacheReadBuffer(range.left);
} }
if (download_offset < file_offset_of_buffer_end) if (file_segment->getCurrentWriteOffset() < file_offset_of_buffer_end)
{ {
/// segment{1} /// segment{1}
/// cache: [_____|___________ /// cache: [_____|___________
/// ^ /// ^
/// download_offset /// current_write_offset
/// requested_range: [__________] /// requested_range: [__________]
/// ^ /// ^
/// file_offset_of_buffer_end /// file_offset_of_buffer_end
assert(file_offset_of_buffer_end > file_segment->getDownloadOffset()); LOG_TEST(log, "Predownload. File segment info: {}", file_segment->getInfoForLog());
bytes_to_predownload = file_offset_of_buffer_end - file_segment->getDownloadOffset(); chassert(file_offset_of_buffer_end > file_segment->getCurrentWriteOffset());
assert(bytes_to_predownload < range.size()); bytes_to_predownload = file_offset_of_buffer_end - file_segment->getCurrentWriteOffset();
chassert(bytes_to_predownload < range.size());
} }
download_offset = file_segment->getDownloadOffset();
can_start_from_cache = download_offset > file_offset_of_buffer_end;
assert(!can_start_from_cache);
read_type = ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE; read_type = ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE;
return getRemoteFSReadBuffer(file_segment, read_type); return getRemoteFSReadBuffer(*file_segment, read_type);
} }
download_state = file_segment->state(); download_state = file_segment->state();
@ -350,10 +351,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
} }
case FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION: case FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION:
{ {
size_t download_offset = file_segment->getDownloadOffset(); if (canStartFromCache(file_offset_of_buffer_end, *file_segment))
bool can_start_from_cache = download_offset > file_offset_of_buffer_end;
if (can_start_from_cache)
{ {
read_type = ReadType::CACHED; read_type = ReadType::CACHED;
return getCacheReadBuffer(range.left); return getCacheReadBuffer(range.left);
@ -364,7 +362,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
log, log,
"Bypassing cache because file segment state is `PARTIALLY_DOWNLOADED_NO_CONTINUATION` and downloaded part already used"); "Bypassing cache because file segment state is `PARTIALLY_DOWNLOADED_NO_CONTINUATION` and downloaded part already used");
read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE; read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE;
return getRemoteFSReadBuffer(file_segment, read_type); return getRemoteFSReadBuffer(*file_segment, read_type);
} }
} }
} }
@ -374,8 +372,8 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil
CachedOnDiskReadBufferFromFile::ImplementationBufferPtr CachedOnDiskReadBufferFromFile::ImplementationBufferPtr
CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_segment) CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_segment)
{ {
assert(!file_segment->isDownloader()); chassert(!file_segment->isDownloader());
assert(file_offset_of_buffer_end >= file_segment->range().left); chassert(file_offset_of_buffer_end >= file_segment->range().left);
auto range = file_segment->range(); auto range = file_segment->range();
bytes_to_predownload = 0; bytes_to_predownload = 0;
@ -389,10 +387,10 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_se
ProfileEvents::FileSegmentWaitReadBufferMicroseconds, watch.elapsedMicroseconds()); ProfileEvents::FileSegmentWaitReadBufferMicroseconds, watch.elapsedMicroseconds());
[[maybe_unused]] auto download_current_segment = read_type == ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE; [[maybe_unused]] auto download_current_segment = read_type == ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE;
assert(download_current_segment == file_segment->isDownloader()); chassert(download_current_segment == file_segment->isDownloader());
assert(file_segment->range() == range); chassert(file_segment->range() == range);
assert(file_offset_of_buffer_end >= range.left && file_offset_of_buffer_end <= range.right); chassert(file_offset_of_buffer_end >= range.left && file_offset_of_buffer_end <= range.right);
LOG_TEST( LOG_TEST(
log, log,
@ -441,12 +439,12 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_se
} }
case ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE: case ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE:
{ {
assert(file_segment->isDownloader()); chassert(file_segment->isDownloader());
if (bytes_to_predownload) if (bytes_to_predownload)
{ {
size_t download_offset = file_segment->getDownloadOffset(); size_t current_write_offset = file_segment->getCurrentWriteOffset();
read_buffer_for_file_segment->seek(download_offset, SEEK_SET); read_buffer_for_file_segment->seek(current_write_offset, SEEK_SET);
} }
else else
{ {
@ -456,18 +454,15 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_se
assert(static_cast<size_t>(read_buffer_for_file_segment->getFileOffsetOfBufferEnd()) == file_offset_of_buffer_end); assert(static_cast<size_t>(read_buffer_for_file_segment->getFileOffsetOfBufferEnd()) == file_offset_of_buffer_end);
} }
auto download_offset = file_segment->getDownloadOffset(); auto current_write_offset = file_segment->getCurrentWriteOffset();
if (download_offset != static_cast<size_t>(read_buffer_for_file_segment->getPosition())) if (current_write_offset != static_cast<size_t>(read_buffer_for_file_segment->getPosition()))
{ {
throw Exception( throw Exception(
ErrorCodes::LOGICAL_ERROR, ErrorCodes::LOGICAL_ERROR,
"Buffer's offsets mismatch; cached buffer offset: {}, download_offset: {}, " "Buffer's offsets mismatch. Cached buffer offset: {}, current_write_offset: {} implementation buffer offset: {}, "
"position: {}, implementation buffer remaining read range: {}, file segment info: {}", "implementation buffer remaining range: {}, file segment info: {}",
file_offset_of_buffer_end, file_offset_of_buffer_end, current_write_offset, read_buffer_for_file_segment->getPosition(),
download_offset, read_buffer_for_file_segment->getRemainingReadRange().toString(), file_segment->getInfoForLog());
read_buffer_for_file_segment->getPosition(),
read_buffer_for_file_segment->getRemainingReadRange().toString(),
file_segment->getInfoForLog());
} }
break; break;
@ -488,7 +483,7 @@ bool CachedOnDiskReadBufferFromFile::completeFileSegmentAndGetNext()
auto & file_segment = *file_segment_it; auto & file_segment = *file_segment_it;
[[maybe_unused]] const auto & range = file_segment->range(); [[maybe_unused]] const auto & range = file_segment->range();
assert(file_offset_of_buffer_end > range.right); chassert(file_offset_of_buffer_end > range.right);
LOG_TEST( LOG_TEST(
log, log,
@ -499,9 +494,7 @@ bool CachedOnDiskReadBufferFromFile::completeFileSegmentAndGetNext()
/// Do not hold pointer to file segment if it is not needed anymore /// Do not hold pointer to file segment if it is not needed anymore
/// so can become releasable and can be evicted from cache. /// so can become releasable and can be evicted from cache.
/// If the status of filesegment state is SKIP_CACHE, it will not be deleted. file_segment->completeWithoutState();
/// It will be deleted from the cache when the holder is destructed.
if ((*file_segment_it)->state() != FileSegment::State::SKIP_CACHE)
file_segments_holder->file_segments.erase(file_segment_it); file_segments_holder->file_segments.erase(file_segment_it);
if (current_file_segment_it == file_segments_holder->file_segments.end()) if (current_file_segment_it == file_segments_holder->file_segments.end())
@ -545,8 +538,8 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment)
/// download from offset a'' < a', but return buffer from offset a'. /// download from offset a'' < a', but return buffer from offset a'.
LOG_TEST(log, "Bytes to predownload: {}, caller_id: {}", bytes_to_predownload, FileSegment::getCallerId()); LOG_TEST(log, "Bytes to predownload: {}, caller_id: {}", bytes_to_predownload, FileSegment::getCallerId());
assert(implementation_buffer->getFileOffsetOfBufferEnd() == file_segment->getDownloadOffset()); chassert(implementation_buffer->getFileOffsetOfBufferEnd() == file_segment->getCurrentWriteOffset());
size_t current_offset = file_segment->getDownloadOffset(); size_t current_offset = file_segment->getCurrentWriteOffset();
const auto & current_range = file_segment->range(); const auto & current_range = file_segment->range();
while (true) while (true)
@ -572,7 +565,7 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment)
"current download offset: {}, expected: {}, eof: {}", "current download offset: {}, expected: {}, eof: {}",
bytes_to_predownload, bytes_to_predownload,
current_range.toString(), current_range.toString(),
file_segment->getDownloadOffset(), file_segment->getCurrentWriteOffset(),
file_offset_of_buffer_end, file_offset_of_buffer_end,
implementation_buffer->eof()); implementation_buffer->eof());
@ -582,19 +575,21 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment)
{ {
nextimpl_working_buffer_offset = implementation_buffer->offset(); nextimpl_working_buffer_offset = implementation_buffer->offset();
auto download_offset = file_segment->getDownloadOffset(); auto current_write_offset = file_segment->getCurrentWriteOffset();
if (download_offset != static_cast<size_t>(implementation_buffer->getPosition()) if (current_write_offset != static_cast<size_t>(implementation_buffer->getPosition())
|| download_offset != file_offset_of_buffer_end) || current_write_offset != file_offset_of_buffer_end)
{
throw Exception( throw Exception(
ErrorCodes::LOGICAL_ERROR, ErrorCodes::LOGICAL_ERROR,
"Buffer's offsets mismatch after predownloading; download offset: {}, " "Buffer's offsets mismatch after predownloading; download offset: {}, "
"cached buffer offset: {}, implementation buffer offset: {}, " "cached buffer offset: {}, implementation buffer offset: {}, "
"file segment info: {}", "file segment info: {}",
download_offset, current_write_offset,
file_offset_of_buffer_end, file_offset_of_buffer_end,
implementation_buffer->getPosition(), implementation_buffer->getPosition(),
file_segment->getInfoForLog()); file_segment->getInfoForLog());
} }
}
break; break;
} }
@ -609,7 +604,7 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment)
{ {
LOG_TEST(log, "Left to predownload: {}, buffer size: {}", bytes_to_predownload, current_impl_buffer_size); LOG_TEST(log, "Left to predownload: {}, buffer size: {}", bytes_to_predownload, current_impl_buffer_size);
assert(file_segment->getDownloadOffset() == static_cast<size_t>(implementation_buffer->getPosition())); chassert(file_segment->getCurrentWriteOffset() == static_cast<size_t>(implementation_buffer->getPosition()));
bool success = writeCache(implementation_buffer->buffer().begin(), current_predownload_size, current_offset, *file_segment); bool success = writeCache(implementation_buffer->buffer().begin(), current_predownload_size, current_offset, *file_segment);
if (success) if (success)
@ -635,7 +630,7 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment)
/// segment{1} /// segment{1}
/// cache: [_____|___________ /// cache: [_____|___________
/// ^ /// ^
/// download_offset /// current_write_offset
/// requested_range: [__________] /// requested_range: [__________]
/// ^ /// ^
/// file_offset_of_buffer_end /// file_offset_of_buffer_end
@ -649,17 +644,18 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment)
bytes_to_predownload = 0; bytes_to_predownload = 0;
file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
LOG_TEST(log, "Bypassing cache because space reservation failed"); LOG_TEST(log, "Bypassing cache because for {}", file_segment->getInfoForLog());
read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE; read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE;
swap(*implementation_buffer); swap(*implementation_buffer);
resetWorkingBuffer(); resetWorkingBuffer();
implementation_buffer = getRemoteFSReadBuffer(file_segment, read_type); implementation_buffer = getRemoteFSReadBuffer(*file_segment, read_type);
swap(*implementation_buffer); swap(*implementation_buffer);
implementation_buffer->setReadUntilPosition(current_range.right + 1); /// [..., range.right] implementation_buffer->setReadUntilPosition(file_segment->range().right + 1); /// [..., range.right]
implementation_buffer->seek(file_offset_of_buffer_end, SEEK_SET); implementation_buffer->seek(file_offset_of_buffer_end, SEEK_SET);
LOG_TEST( LOG_TEST(
@ -680,8 +676,8 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded()
auto current_read_range = file_segment->range(); auto current_read_range = file_segment->range();
auto current_state = file_segment->state(); auto current_state = file_segment->state();
assert(current_read_range.left <= file_offset_of_buffer_end); chassert(current_read_range.left <= file_offset_of_buffer_end);
assert(!file_segment->isDownloader()); chassert(!file_segment->isDownloader());
if (file_offset_of_buffer_end > current_read_range.right) if (file_offset_of_buffer_end > current_read_range.right)
{ {
@ -695,13 +691,15 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded()
/// segment{k} /// segment{k}
/// cache: [______|___________ /// cache: [______|___________
/// ^ /// ^
/// download_offset /// current_write_offset
/// requested_range: [__________] /// requested_range: [__________]
/// ^ /// ^
/// file_offset_of_buffer_end /// file_offset_of_buffer_end
size_t download_offset = file_segment->getDownloadOffset(); auto current_write_offset = file_segment->getCurrentWriteOffset();
bool cached_part_is_finished = download_offset == file_offset_of_buffer_end; bool cached_part_is_finished = current_write_offset == file_offset_of_buffer_end;
LOG_TEST(log, "Current write offset: {}, file offset of buffer end: {}", current_write_offset, file_offset_of_buffer_end);
if (cached_part_is_finished) if (cached_part_is_finished)
{ {
@ -710,12 +708,12 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded()
return true; return true;
} }
else if (download_offset < file_offset_of_buffer_end) else if (current_write_offset < file_offset_of_buffer_end)
{ {
throw Exception( throw Exception(
ErrorCodes::LOGICAL_ERROR, ErrorCodes::LOGICAL_ERROR,
"Expected {} >= {} ({})", "Expected {} >= {} ({})",
download_offset, file_offset_of_buffer_end, getInfoForLog()); current_write_offset, file_offset_of_buffer_end, getInfoForLog());
} }
} }
@ -725,7 +723,7 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded()
* ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE means that on previous getImplementationBuffer() call * ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE means that on previous getImplementationBuffer() call
* current buffer successfully called file_segment->getOrSetDownloader() and became a downloader * current buffer successfully called file_segment->getOrSetDownloader() and became a downloader
* for this file segment. However, the downloader's term has a lifespan of 1 nextImpl() call, * for this file segment. However, the downloader's term has a lifespan of 1 nextImpl() call,
* e.g. downloader reads buffer_size byte and calls completeBatchAndResetDownloader() and some other * e.g. downloader reads buffer_size byte and calls completePartAndResetDownloader() and some other
* thread can become a downloader if it calls getOrSetDownloader() faster. * thread can become a downloader if it calls getOrSetDownloader() faster.
* *
* So downloader is committed to download only buffer_size bytes and then is not a downloader anymore, * So downloader is committed to download only buffer_size bytes and then is not a downloader anymore,
@ -817,11 +815,11 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
if (need_complete_file_segment) if (need_complete_file_segment)
{ {
LOG_TEST(log, "Resetting downloader {} from scope exit", file_segment->getDownloader()); LOG_TEST(log, "Resetting downloader {} from scope exit", file_segment->getDownloader());
file_segment->completeBatchAndResetDownloader(); file_segment->completePartAndResetDownloader();
} }
} }
assert(!file_segment->isDownloader()); chassert(!file_segment->isDownloader());
} }
catch (...) catch (...)
{ {
@ -845,7 +843,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
(*current_file_segment_it)->incrementHitsCount(); (*current_file_segment_it)->incrementHitsCount();
} }
assert(!internal_buffer.empty()); chassert(!internal_buffer.empty());
swap(*implementation_buffer); swap(*implementation_buffer);
@ -854,15 +852,14 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
LOG_TEST( LOG_TEST(
log, log,
"Current segment: {}, downloader: {}, current count: {}, position: {}, read range: {}", "Current count: {}, position: {}, read range: {}, file segment: {}",
current_read_range.toString(),
file_segment->getDownloader(),
implementation_buffer->count(), implementation_buffer->count(),
implementation_buffer->getPosition(), implementation_buffer->getPosition(),
implementation_buffer->getRemainingReadRange().toString()); implementation_buffer->getRemainingReadRange().toString(),
file_segment->getInfoForLog());
assert(current_read_range.left <= file_offset_of_buffer_end); chassert(current_read_range.left <= file_offset_of_buffer_end);
assert(current_read_range.right >= file_offset_of_buffer_end); chassert(current_read_range.right >= file_offset_of_buffer_end);
bool result = false; bool result = false;
size_t size = 0; size_t size = 0;
@ -939,24 +936,26 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
{ {
if (download_current_segment) if (download_current_segment)
{ {
assert(file_offset_of_buffer_end + size - 1 <= file_segment->range().right); chassert(file_offset_of_buffer_end + size - 1 <= file_segment->range().right);
bool success = file_segment->reserve(size); bool success = file_segment->reserve(size);
if (success) if (success)
{ {
assert(file_segment->getDownloadOffset() == static_cast<size_t>(implementation_buffer->getPosition())); chassert(file_segment->getCurrentWriteOffset() == static_cast<size_t>(implementation_buffer->getPosition()));
success = writeCache(implementation_buffer->position(), size, file_offset_of_buffer_end, *file_segment); success = writeCache(implementation_buffer->position(), size, file_offset_of_buffer_end, *file_segment);
if (success) if (success)
{ {
assert(file_segment->getDownloadOffset() <= file_segment->range().right + 1); chassert(file_segment->getCurrentWriteOffset() <= file_segment->range().right + 1);
assert( chassert(
std::next(current_file_segment_it) == file_segments_holder->file_segments.end() std::next(current_file_segment_it) == file_segments_holder->file_segments.end()
|| file_segment->getDownloadOffset() == implementation_buffer->getFileOffsetOfBufferEnd()); || file_segment->getCurrentWriteOffset() == implementation_buffer->getFileOffsetOfBufferEnd());
LOG_TEST(log, "Successfully written {} bytes", size);
} }
else else
{ {
assert(file_segment->state() == FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); chassert(file_segment->state() == FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
LOG_TEST(log, "Bypassing cache because writeCache method failed"); LOG_TEST(log, "Bypassing cache because writeCache method failed");
} }
} }
@ -984,7 +983,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
size_t remaining_size_to_read size_t remaining_size_to_read
= std::min(current_read_range.right, read_until_position - 1) - file_offset_of_buffer_end + 1; = std::min(current_read_range.right, read_until_position - 1) - file_offset_of_buffer_end + 1;
size = std::min(size, remaining_size_to_read); size = std::min(size, remaining_size_to_read);
assert(implementation_buffer->buffer().size() >= nextimpl_working_buffer_offset + size); chassert(implementation_buffer->buffer().size() >= nextimpl_working_buffer_offset + size);
implementation_buffer->buffer().resize(nextimpl_working_buffer_offset + size); implementation_buffer->buffer().resize(nextimpl_working_buffer_offset + size);
} }
@ -996,15 +995,15 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
current_file_segment_counters.increment(ProfileEvents::FileSegmentUsedBytes, available()); current_file_segment_counters.increment(ProfileEvents::FileSegmentUsedBytes, available());
if (download_current_segment) if (download_current_segment)
file_segment->completeBatchAndResetDownloader(); file_segment->completePartAndResetDownloader();
assert(!file_segment->isDownloader()); chassert(!file_segment->isDownloader());
LOG_TEST( LOG_TEST(
log, log,
"Key: {}. Returning with {} bytes, buffer position: {} (offset: {}, predownloaded: {}), " "Key: {}. Returning with {} bytes, buffer position: {} (offset: {}, predownloaded: {}), "
"buffer available: {}, current range: {}, current offset: {}, file segment state: {}, " "buffer available: {}, current range: {}, current offset: {}, file segment state: {}, "
"download offset: {}, read_type: {}, reading until position: {}, started with offset: {}, " "current write offset: {}, read_type: {}, reading until position: {}, started with offset: {}, "
"remaining ranges: {}", "remaining ranges: {}",
getHexUIntLowercase(cache_key), getHexUIntLowercase(cache_key),
working_buffer.size(), working_buffer.size(),
@ -1015,7 +1014,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
current_read_range.toString(), current_read_range.toString(),
file_offset_of_buffer_end, file_offset_of_buffer_end,
FileSegment::stateToString(file_segment->state()), FileSegment::stateToString(file_segment->state()),
file_segment->getDownloadOffset(), file_segment->getCurrentWriteOffset(),
toString(read_type), toString(read_type),
read_until_position, read_until_position,
first_offset, first_offset,

View File

@ -80,7 +80,7 @@ private:
void assertCorrectness() const; void assertCorrectness() const;
std::shared_ptr<ReadBufferFromFileBase> getRemoteFSReadBuffer(FileSegmentPtr & file_segment, ReadType read_type_); std::shared_ptr<ReadBufferFromFileBase> getRemoteFSReadBuffer(FileSegment & file_segment, ReadType read_type_);
size_t getTotalSizeToRead(); size_t getTotalSizeToRead();
@ -90,6 +90,8 @@ private:
bool writeCache(char * data, size_t size, size_t offset, FileSegment & file_segment); bool writeCache(char * data, size_t size, size_t offset, FileSegment & file_segment);
static bool canStartFromCache(size_t current_offset, const FileSegment & file_segment);
Poco::Logger * log; Poco::Logger * log;
FileCache::Key cache_key; FileCache::Key cache_key;
String source_file_path; String source_file_path;

View File

@ -77,7 +77,7 @@ void CachedOnDiskWriteBufferFromFile::nextImpl()
void CachedOnDiskWriteBufferFromFile::cacheData(char * data, size_t size) void CachedOnDiskWriteBufferFromFile::cacheData(char * data, size_t size)
{ {
if (stop_caching) if (cache_in_error_state_or_disabled)
return; return;
if (!cache_writer) if (!cache_writer)
@ -88,14 +88,13 @@ void CachedOnDiskWriteBufferFromFile::cacheData(char * data, size_t size)
Stopwatch watch(CLOCK_MONOTONIC); Stopwatch watch(CLOCK_MONOTONIC);
cache_in_error_state_or_disabled = true;
try try
{ {
if (!cache_writer->write(data, size, current_download_offset, is_persistent_cache_file)) if (!cache_writer->write(data, size, current_download_offset, is_persistent_cache_file))
{ {
LOG_INFO(log, "Write-through cache is stopped as cache limit is reached and nothing can be evicted"); LOG_INFO(log, "Write-through cache is stopped as cache limit is reached and nothing can be evicted");
/// No space left, disable caching.
stop_caching = true;
return; return;
} }
} }
@ -122,6 +121,8 @@ void CachedOnDiskWriteBufferFromFile::cacheData(char * data, size_t size)
current_file_segment_counters.increment( current_file_segment_counters.increment(
ProfileEvents::FileSegmentWriteMicroseconds, watch.elapsedMicroseconds()); ProfileEvents::FileSegmentWriteMicroseconds, watch.elapsedMicroseconds());
cache_in_error_state_or_disabled = false;
} }
void CachedOnDiskWriteBufferFromFile::appendFilesystemCacheLog(const FileSegment & file_segment) void CachedOnDiskWriteBufferFromFile::appendFilesystemCacheLog(const FileSegment & file_segment)

View File

@ -51,7 +51,7 @@ private:
bool enable_cache_log; bool enable_cache_log;
std::shared_ptr<FilesystemCacheLog> cache_log; std::shared_ptr<FilesystemCacheLog> cache_log;
bool stop_caching = false; bool cache_in_error_state_or_disabled = false;
ProfileEvents::Counters current_file_segment_counters; ProfileEvents::Counters current_file_segment_counters;
std::unique_ptr<FileSegmentRangeWriter> cache_writer; std::unique_ptr<FileSegmentRangeWriter> cache_writer;

View File

@ -76,6 +76,8 @@ struct ReadSettings
/// For 'pread_threadpool' method. Lower is more priority. /// For 'pread_threadpool' method. Lower is more priority.
size_t priority = 0; size_t priority = 0;
bool load_marks_asynchronously = true;
size_t remote_fs_read_max_backoff_ms = 10000; size_t remote_fs_read_max_backoff_ms = 10000;
size_t remote_fs_read_backoff_max_tries = 4; size_t remote_fs_read_backoff_max_tries = 4;

View File

@ -26,6 +26,7 @@
#include <Common/config.h> #include <Common/config.h>
#include <Common/config_version.h> #include <Common/config_version.h>
#include <filesystem>
namespace ProfileEvents namespace ProfileEvents
{ {
@ -346,13 +347,29 @@ namespace detail
non_retriable_errors.begin(), non_retriable_errors.end(), [&](const auto status) { return http_status != status; }); non_retriable_errors.begin(), non_retriable_errors.end(), [&](const auto status) { return http_status != status; });
} }
Poco::URI getUriAfterRedirect(const Poco::URI & prev_uri, Poco::Net::HTTPResponse & response)
{
auto location = response.get("Location");
auto location_uri = Poco::URI(location);
if (!location_uri.isRelative())
return location_uri;
/// Location header contains relative path. So we need to concatenate it
/// with path from the original URI and normalize it.
auto path = std::filesystem::weakly_canonical(std::filesystem::path(prev_uri.getPath()) / location);
location_uri = prev_uri;
location_uri.setPath(path);
return location_uri;
}
void callWithRedirects(Poco::Net::HTTPResponse & response, const String & method_, bool throw_on_all_errors = false) void callWithRedirects(Poco::Net::HTTPResponse & response, const String & method_, bool throw_on_all_errors = false)
{ {
call(response, method_, throw_on_all_errors); call(response, method_, throw_on_all_errors);
Poco::URI prev_uri = uri;
while (isRedirect(response.getStatus())) while (isRedirect(response.getStatus()))
{ {
Poco::URI uri_redirect(response.get("Location")); Poco::URI uri_redirect = getUriAfterRedirect(prev_uri, response);
prev_uri = uri_redirect;
if (remote_host_filter) if (remote_host_filter)
remote_host_filter->checkURL(uri_redirect); remote_host_filter->checkURL(uri_redirect);
@ -408,7 +425,7 @@ namespace detail
while (isRedirect(response.getStatus())) while (isRedirect(response.getStatus()))
{ {
Poco::URI uri_redirect(response.get("Location")); Poco::URI uri_redirect = getUriAfterRedirect(saved_uri_redirect.value_or(uri), response);
if (remote_host_filter) if (remote_host_filter)
remote_host_filter->checkURL(uri_redirect); remote_host_filter->checkURL(uri_redirect);

View File

@ -122,7 +122,6 @@ void FileCache::initialize()
fs::create_directories(cache_base_path); fs::create_directories(cache_base_path);
} }
status_file = make_unique<StatusFile>(fs::path(cache_base_path) / "status", StatusFile::write_full_info);
is_initialized = true; is_initialized = true;
} }
} }
@ -258,7 +257,7 @@ FileSegments FileCache::splitRangeIntoCells(
size_t offset, size_t offset,
size_t size, size_t size,
FileSegment::State state, FileSegment::State state,
bool is_persistent, const CreateFileSegmentSettings & settings,
std::lock_guard<std::mutex> & cache_lock) std::lock_guard<std::mutex> & cache_lock)
{ {
assert(size > 0); assert(size > 0);
@ -275,7 +274,7 @@ FileSegments FileCache::splitRangeIntoCells(
current_cell_size = std::min(remaining_size, max_file_segment_size); current_cell_size = std::min(remaining_size, max_file_segment_size);
remaining_size -= current_cell_size; remaining_size -= current_cell_size;
auto * cell = addCell(key, current_pos, current_cell_size, state, is_persistent, cache_lock); auto * cell = addCell(key, current_pos, current_cell_size, state, settings, cache_lock);
if (cell) if (cell)
file_segments.push_back(cell->file_segment); file_segments.push_back(cell->file_segment);
assert(cell); assert(cell);
@ -292,7 +291,7 @@ void FileCache::fillHolesWithEmptyFileSegments(
const Key & key, const Key & key,
const FileSegment::Range & range, const FileSegment::Range & range,
bool fill_with_detached_file_segments, bool fill_with_detached_file_segments,
bool is_persistent, const CreateFileSegmentSettings & settings,
std::lock_guard<std::mutex> & cache_lock) std::lock_guard<std::mutex> & cache_lock)
{ {
/// There are segments [segment1, ..., segmentN] /// There are segments [segment1, ..., segmentN]
@ -339,16 +338,16 @@ void FileCache::fillHolesWithEmptyFileSegments(
if (fill_with_detached_file_segments) if (fill_with_detached_file_segments)
{ {
auto file_segment = std::make_shared<FileSegment>(current_pos, hole_size, key, this, FileSegment::State::EMPTY); auto file_segment = std::make_shared<FileSegment>(current_pos, hole_size, key, this, FileSegment::State::EMPTY, settings);
{ {
std::lock_guard segment_lock(file_segment->mutex); std::unique_lock segment_lock(file_segment->mutex);
file_segment->markAsDetached(segment_lock); file_segment->detachAssumeStateFinalized(segment_lock);
} }
file_segments.insert(it, file_segment); file_segments.insert(it, file_segment);
} }
else else
{ {
file_segments.splice(it, splitRangeIntoCells(key, current_pos, hole_size, FileSegment::State::EMPTY, is_persistent, cache_lock)); file_segments.splice(it, splitRangeIntoCells(key, current_pos, hole_size, FileSegment::State::EMPTY, settings, cache_lock));
} }
current_pos = segment_range.right + 1; current_pos = segment_range.right + 1;
@ -366,22 +365,23 @@ void FileCache::fillHolesWithEmptyFileSegments(
if (fill_with_detached_file_segments) if (fill_with_detached_file_segments)
{ {
auto file_segment = std::make_shared<FileSegment>(current_pos, hole_size, key, this, FileSegment::State::EMPTY); auto file_segment = std::make_shared<FileSegment>(current_pos, hole_size, key, this, FileSegment::State::EMPTY, settings);
{ {
std::lock_guard segment_lock(file_segment->mutex); std::unique_lock segment_lock(file_segment->mutex);
file_segment->markAsDetached(segment_lock); file_segment->detachAssumeStateFinalized(segment_lock);
} }
file_segments.insert(file_segments.end(), file_segment); file_segments.insert(file_segments.end(), file_segment);
} }
else else
{ {
file_segments.splice( file_segments.splice(
file_segments.end(), splitRangeIntoCells(key, current_pos, hole_size, FileSegment::State::EMPTY, is_persistent, cache_lock)); file_segments.end(),
splitRangeIntoCells(key, current_pos, hole_size, FileSegment::State::EMPTY, settings, cache_lock));
} }
} }
} }
FileSegmentsHolder FileCache::getOrSet(const Key & key, size_t offset, size_t size, bool is_persistent) FileSegmentsHolder FileCache::getOrSet(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings)
{ {
std::lock_guard cache_lock(mutex); std::lock_guard cache_lock(mutex);
@ -398,11 +398,11 @@ FileSegmentsHolder FileCache::getOrSet(const Key & key, size_t offset, size_t si
if (file_segments.empty()) if (file_segments.empty())
{ {
file_segments = splitRangeIntoCells(key, offset, size, FileSegment::State::EMPTY, is_persistent, cache_lock); file_segments = splitRangeIntoCells(key, offset, size, FileSegment::State::EMPTY, settings, cache_lock);
} }
else else
{ {
fillHolesWithEmptyFileSegments(file_segments, key, range, /* fill_with_detached */false, is_persistent, cache_lock); fillHolesWithEmptyFileSegments(file_segments, key, range, /* fill_with_detached */false, settings, cache_lock);
} }
assert(!file_segments.empty()); assert(!file_segments.empty());
@ -426,16 +426,17 @@ FileSegmentsHolder FileCache::get(const Key & key, size_t offset, size_t size)
if (file_segments.empty()) if (file_segments.empty())
{ {
auto file_segment = std::make_shared<FileSegment>(offset, size, key, this, FileSegment::State::EMPTY); auto file_segment = std::make_shared<FileSegment>(
offset, size, key, this, FileSegment::State::EMPTY, CreateFileSegmentSettings{});
{ {
std::lock_guard segment_lock(file_segment->mutex); std::unique_lock segment_lock(file_segment->mutex);
file_segment->markAsDetached(segment_lock); file_segment->detachAssumeStateFinalized(segment_lock);
} }
file_segments = { file_segment }; file_segments = { file_segment };
} }
else else
{ {
fillHolesWithEmptyFileSegments(file_segments, key, range, /* fill_with_detached */true, /* is_persistent */false, cache_lock); fillHolesWithEmptyFileSegments(file_segments, key, range, /* fill_with_detached */true, {}, cache_lock);
} }
return FileSegmentsHolder(std::move(file_segments)); return FileSegmentsHolder(std::move(file_segments));
@ -443,7 +444,7 @@ FileSegmentsHolder FileCache::get(const Key & key, size_t offset, size_t size)
FileCache::FileSegmentCell * FileCache::addCell( FileCache::FileSegmentCell * FileCache::addCell(
const Key & key, size_t offset, size_t size, const Key & key, size_t offset, size_t size,
FileSegment::State state, bool is_persistent, FileSegment::State state, const CreateFileSegmentSettings & settings,
std::lock_guard<std::mutex> & cache_lock) std::lock_guard<std::mutex> & cache_lock)
{ {
/// Create a file segment cell and put it in `files` map by [key][offset]. /// Create a file segment cell and put it in `files` map by [key][offset].
@ -475,18 +476,23 @@ FileCache::FileSegmentCell * FileCache::addCell(
stash_records.erase({remove_priority_iter->key(), remove_priority_iter->offset()}); stash_records.erase({remove_priority_iter->key(), remove_priority_iter->offset()});
remove_priority_iter->removeAndGetNext(cache_lock); remove_priority_iter->removeAndGetNext(cache_lock);
} }
/// For segments that do not reach the download threshold, we do not download them, but directly read them
/// For segments that do not reach the download threshold,
/// we do not download them, but directly read them
result_state = FileSegment::State::SKIP_CACHE; result_state = FileSegment::State::SKIP_CACHE;
} }
else else
{ {
auto priority_iter = record->second; auto priority_iter = record->second;
priority_iter->use(cache_lock); priority_iter->use(cache_lock);
result_state = priority_iter->hits() >= enable_cache_hits_threshold ? FileSegment::State::EMPTY : FileSegment::State::SKIP_CACHE;
result_state = priority_iter->hits() >= enable_cache_hits_threshold
? FileSegment::State::EMPTY
: FileSegment::State::SKIP_CACHE;
} }
} }
return std::make_shared<FileSegment>(offset, size, key, this, result_state, is_persistent); return std::make_shared<FileSegment>(offset, size, key, this, result_state, settings);
}; };
FileSegmentCell cell(skip_or_download(), this, cache_lock); FileSegmentCell cell(skip_or_download(), this, cache_lock);
@ -495,6 +501,7 @@ FileCache::FileSegmentCell * FileCache::addCell(
if (offsets.empty()) if (offsets.empty())
{ {
auto key_path = getPathInLocalCache(key); auto key_path = getPathInLocalCache(key);
if (!fs::exists(key_path)) if (!fs::exists(key_path))
fs::create_directories(key_path); fs::create_directories(key_path);
} }
@ -513,7 +520,7 @@ FileSegmentPtr FileCache::createFileSegmentForDownload(
const Key & key, const Key & key,
size_t offset, size_t offset,
size_t size, size_t size,
bool is_persistent, const CreateFileSegmentSettings & settings,
std::lock_guard<std::mutex> & cache_lock) std::lock_guard<std::mutex> & cache_lock)
{ {
#ifndef NDEBUG #ifndef NDEBUG
@ -530,7 +537,7 @@ FileSegmentPtr FileCache::createFileSegmentForDownload(
"Cache cell already exists for key `{}` and offset {}", "Cache cell already exists for key `{}` and offset {}",
key.toString(), offset); key.toString(), offset);
cell = addCell(key, offset, size, FileSegment::State::EMPTY, is_persistent, cache_lock); cell = addCell(key, offset, size, FileSegment::State::EMPTY, settings, cache_lock);
if (!cell) if (!cell)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to add a new cell for download"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to add a new cell for download");
@ -542,18 +549,21 @@ bool FileCache::tryReserve(const Key & key, size_t offset, size_t size, std::loc
{ {
auto query_context = enable_filesystem_query_cache_limit ? getCurrentQueryContext(cache_lock) : nullptr; auto query_context = enable_filesystem_query_cache_limit ? getCurrentQueryContext(cache_lock) : nullptr;
if (!query_context) if (!query_context)
{
return tryReserveForMainList(key, offset, size, nullptr, cache_lock); return tryReserveForMainList(key, offset, size, nullptr, cache_lock);
}
/// The maximum cache capacity of the request is not reached, thus the /// The maximum cache capacity of the request is not reached, thus the
//// cache block is evicted from the main LRU queue by tryReserveForMainList(). //// cache block is evicted from the main LRU queue by tryReserveForMainList().
else if (query_context->getCacheSize() + size <= query_context->getMaxCacheSize()) else if (query_context->getCacheSize() + size <= query_context->getMaxCacheSize())
{
return tryReserveForMainList(key, offset, size, query_context, cache_lock); return tryReserveForMainList(key, offset, size, query_context, cache_lock);
}
/// When skip_download_if_exceeds_query_cache is true, there is no need /// When skip_download_if_exceeds_query_cache is true, there is no need
/// to evict old data, skip the cache and read directly from remote fs. /// to evict old data, skip the cache and read directly from remote fs.
else if (query_context->isSkipDownloadIfExceed()) else if (query_context->isSkipDownloadIfExceed())
{
return false; return false;
}
/// The maximum cache size of the query is reached, the cache will be /// The maximum cache size of the query is reached, the cache will be
/// evicted from the history cache accessed by the current query. /// evicted from the history cache accessed by the current query.
else else
@ -833,7 +843,7 @@ void FileCache::removeIfExists(const Key & key)
auto file_segment = cell->file_segment; auto file_segment = cell->file_segment;
if (file_segment) if (file_segment)
{ {
std::lock_guard<std::mutex> segment_lock(file_segment->mutex); std::unique_lock<std::mutex> segment_lock(file_segment->mutex);
file_segment->detach(cache_lock, segment_lock); file_segment->detach(cache_lock, segment_lock);
remove(file_segment->key(), file_segment->offset(), cache_lock, segment_lock); remove(file_segment->key(), file_segment->offset(), cache_lock, segment_lock);
} }
@ -863,9 +873,11 @@ void FileCache::removeIfReleasable()
auto * cell = getCell(key, offset, cache_lock); auto * cell = getCell(key, offset, cache_lock);
if (!cell) if (!cell)
{
throw Exception( throw Exception(
ErrorCodes::LOGICAL_ERROR, ErrorCodes::LOGICAL_ERROR,
"Cache is in inconsistent state: LRU queue contains entries with no cache cell"); "Cache is in inconsistent state: LRU queue contains entries with no cache cell");
}
if (cell->releasable()) if (cell->releasable())
{ {
@ -880,7 +892,7 @@ void FileCache::removeIfReleasable()
for (auto & file_segment : to_remove) for (auto & file_segment : to_remove)
{ {
std::lock_guard segment_lock(file_segment->mutex); std::unique_lock segment_lock(file_segment->mutex);
file_segment->detach(cache_lock, segment_lock); file_segment->detach(cache_lock, segment_lock);
remove(file_segment->key(), file_segment->offset(), cache_lock, segment_lock); remove(file_segment->key(), file_segment->offset(), cache_lock, segment_lock);
} }
@ -896,13 +908,13 @@ void FileCache::removeIfReleasable()
void FileCache::remove(FileSegmentPtr file_segment, std::lock_guard<std::mutex> & cache_lock) void FileCache::remove(FileSegmentPtr file_segment, std::lock_guard<std::mutex> & cache_lock)
{ {
std::lock_guard segment_lock(file_segment->mutex); std::unique_lock segment_lock(file_segment->mutex);
remove(file_segment->key(), file_segment->offset(), cache_lock, segment_lock); remove(file_segment->key(), file_segment->offset(), cache_lock, segment_lock);
} }
void FileCache::remove( void FileCache::remove(
Key key, size_t offset, Key key, size_t offset,
std::lock_guard<std::mutex> & cache_lock, std::lock_guard<std::mutex> & /* segment_lock */) std::lock_guard<std::mutex> & cache_lock, std::unique_lock<std::mutex> & /* segment_lock */)
{ {
LOG_DEBUG(log, "Remove from cache. Key: {}, offset: {}", key.toString(), offset); LOG_DEBUG(log, "Remove from cache. Key: {}, offset: {}", key.toString(), offset);
@ -976,7 +988,7 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard<std::mutex> & cache_lock
{ {
if (!key_it->is_directory()) if (!key_it->is_directory())
{ {
LOG_DEBUG(log, "Unexpected file {} (not a directory), will skip it", key_it->path().string()); LOG_DEBUG(log, "Unexpected file: {}. Expected a directory", key_it->path().string());
continue; continue;
} }
@ -1012,7 +1024,10 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard<std::mutex> & cache_lock
if (tryReserve(key, offset, size, cache_lock)) if (tryReserve(key, offset, size, cache_lock))
{ {
auto * cell = addCell(key, offset, size, FileSegment::State::DOWNLOADED, is_persistent, cache_lock); auto * cell = addCell(
key, offset, size, FileSegment::State::DOWNLOADED,
CreateFileSegmentSettings{ .is_persistent = is_persistent }, cache_lock);
if (cell) if (cell)
queue_entries.emplace_back(cell->queue_iterator, cell->file_segment); queue_entries.emplace_back(cell->queue_iterator, cell->file_segment);
} }
@ -1049,7 +1064,7 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard<std::mutex> & cache_lock
void FileCache::reduceSizeToDownloaded( void FileCache::reduceSizeToDownloaded(
const Key & key, size_t offset, const Key & key, size_t offset,
std::lock_guard<std::mutex> & cache_lock, std::lock_guard<std::mutex> & /* segment_lock */) std::lock_guard<std::mutex> & cache_lock, std::unique_lock<std::mutex> & segment_lock)
{ {
/** /**
* In case file was partially downloaded and it's download cannot be continued * In case file was partially downloaded and it's download cannot be continued
@ -1069,20 +1084,25 @@ void FileCache::reduceSizeToDownloaded(
const auto & file_segment = cell->file_segment; const auto & file_segment = cell->file_segment;
size_t downloaded_size = file_segment->downloaded_size; size_t downloaded_size = file_segment->downloaded_size;
if (downloaded_size == file_segment->range().size()) size_t full_size = file_segment->range().size();
if (downloaded_size == full_size)
{ {
throw Exception( throw Exception(
ErrorCodes::LOGICAL_ERROR, ErrorCodes::LOGICAL_ERROR,
"Nothing to reduce, file segment fully downloaded, key: {}, offset: {}", "Nothing to reduce, file segment fully downloaded: {}",
key.toString(), offset); file_segment->getInfoForLogUnlocked(segment_lock));
} }
cell->file_segment = std::make_shared<FileSegment>(offset, downloaded_size, key, this, FileSegment::State::DOWNLOADED); cell->file_segment = std::make_shared<FileSegment>(
offset, downloaded_size, key, this, FileSegment::State::DOWNLOADED, CreateFileSegmentSettings{});
assert(file_segment->reserved_size == downloaded_size);
} }
bool FileCache::isLastFileSegmentHolder( bool FileCache::isLastFileSegmentHolder(
const Key & key, size_t offset, const Key & key, size_t offset,
std::lock_guard<std::mutex> & cache_lock, std::lock_guard<std::mutex> & /* segment_lock */) std::lock_guard<std::mutex> & cache_lock, std::unique_lock<std::mutex> & /* segment_lock */)
{ {
auto * cell = getCell(key, offset, cache_lock); auto * cell = getCell(key, offset, cache_lock);
@ -1167,7 +1187,8 @@ FileCache::FileSegmentCell::FileSegmentCell(
{ {
case FileSegment::State::DOWNLOADED: case FileSegment::State::DOWNLOADED:
{ {
queue_iterator = cache->main_priority->add(file_segment->key(), file_segment->offset(), file_segment->range().size(), cache_lock); queue_iterator = cache->main_priority->add(
file_segment->key(), file_segment->offset(), file_segment->range().size(), cache_lock);
break; break;
} }
case FileSegment::State::SKIP_CACHE: case FileSegment::State::SKIP_CACHE:
@ -1246,14 +1267,41 @@ void FileCache::assertPriorityCorrectness(std::lock_guard<std::mutex> & cache_lo
ErrorCodes::LOGICAL_ERROR, ErrorCodes::LOGICAL_ERROR,
"Cache is in inconsistent state: LRU queue contains entries with no cache cell (assertCorrectness())"); "Cache is in inconsistent state: LRU queue contains entries with no cache cell (assertCorrectness())");
} }
assert(cell->size() == size);
if (cell->size() != size)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Expected {} == {} size ({})",
cell->size(), size, cell->file_segment->getInfoForLog());
}
total_size += size; total_size += size;
} }
assert(total_size == main_priority->getCacheSize(cache_lock)); assert(total_size == main_priority->getCacheSize(cache_lock));
assert(main_priority->getCacheSize(cache_lock) <= max_size); assert(main_priority->getCacheSize(cache_lock) <= max_size);
assert(main_priority->getElementsNum(cache_lock) <= max_element_size); assert(main_priority->getElementsNum(cache_lock) <= max_element_size);
} }
FileCache::QueryContextHolder::QueryContextHolder(
const String & query_id_,
FileCache * cache_,
FileCache::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)
cache->removeQueryContext(query_id);
}
FileCache::QueryContextPtr FileCache::getCurrentQueryContext(std::lock_guard<std::mutex> & cache_lock) FileCache::QueryContextPtr FileCache::getCurrentQueryContext(std::lock_guard<std::mutex> & cache_lock)
{ {
if (!isQueryInitialized()) if (!isQueryInitialized())
@ -1362,22 +1410,4 @@ void FileCache::QueryContext::use(const Key & key, size_t offset, std::lock_guar
record->second->use(cache_lock); record->second->use(cache_lock);
} }
FileCache::QueryContextHolder::QueryContextHolder(
const String & query_id_,
FileCache * cache_,
FileCache::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)
cache->removeQueryContext(query_id);
}
} }

View File

@ -12,13 +12,14 @@
#include <boost/noncopyable.hpp> #include <boost/noncopyable.hpp>
#include <Core/Types.h> #include <Core/Types.h>
#include <Common/logger_useful.h>
#include <Common/ThreadPool.h>
#include <IO/ReadSettings.h> #include <IO/ReadSettings.h>
#include <Interpreters/Cache/IFileCachePriority.h>
#include <Interpreters/Cache/FileCacheKey.h>
#include <Interpreters/Cache/FileCache_fwd.h> #include <Interpreters/Cache/FileCache_fwd.h>
#include <Interpreters/Cache/FileSegment.h> #include <Interpreters/Cache/FileSegment.h>
#include <Interpreters/Cache/IFileCachePriority.h>
#include <Common/logger_useful.h>
#include <Interpreters/Cache/FileCacheKey.h>
#include <Common/StatusFile.h>
namespace DB namespace DB
{ {
@ -43,7 +44,6 @@ public:
~FileCache() = default; ~FileCache() = default;
/// Restore cache from local filesystem.
void initialize(); void initialize();
const String & getBasePath() const { return cache_base_path; } const String & getBasePath() const { return cache_base_path; }
@ -59,7 +59,7 @@ public:
* As long as pointers to returned file segments are hold * As long as pointers to returned file segments are hold
* it is guaranteed that these file segments are not removed from cache. * it is guaranteed that these file segments are not removed from cache.
*/ */
FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size, bool is_persistent); FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings);
/** /**
* Segments in returned list are ordered in ascending order and represent a full contiguous * Segments in returned list are ordered in ascending order and represent a full contiguous
@ -104,7 +104,7 @@ public:
const Key & key, const Key & key,
size_t offset, size_t offset,
size_t size, size_t size,
bool is_persistent, const CreateFileSegmentSettings & create_settings,
std::lock_guard<std::mutex> & cache_lock); std::lock_guard<std::mutex> & cache_lock);
FileSegments getSnapshot() const; FileSegments getSnapshot() const;
@ -132,21 +132,21 @@ public:
private: private:
String cache_base_path; String cache_base_path;
size_t max_size; const size_t max_size;
size_t max_element_size; const size_t max_element_size;
size_t max_file_segment_size; const size_t max_file_segment_size;
bool allow_persistent_files; const bool allow_persistent_files;
size_t enable_cache_hits_threshold; const size_t enable_cache_hits_threshold;
bool enable_filesystem_query_cache_limit; const bool enable_filesystem_query_cache_limit;
mutable std::mutex mutex;
Poco::Logger * log; Poco::Logger * log;
bool is_initialized = false; bool is_initialized = false;
std::exception_ptr initialization_exception; std::exception_ptr initialization_exception;
std::unique_ptr<StatusFile> status_file;
mutable std::mutex mutex; void assertInitialized(std::lock_guard<std::mutex> & cache_lock) const;
bool tryReserve(const Key & key, size_t offset, size_t size, std::lock_guard<std::mutex> & cache_lock); bool tryReserve(const Key & key, size_t offset, size_t size, std::lock_guard<std::mutex> & cache_lock);
@ -154,7 +154,7 @@ private:
Key key, Key key,
size_t offset, size_t offset,
std::lock_guard<std::mutex> & cache_lock, std::lock_guard<std::mutex> & cache_lock,
std::lock_guard<std::mutex> & segment_lock); std::unique_lock<std::mutex> & segment_lock);
void remove( void remove(
FileSegmentPtr file_segment, FileSegmentPtr file_segment,
@ -164,15 +164,13 @@ private:
const Key & key, const Key & key,
size_t offset, size_t offset,
std::lock_guard<std::mutex> & cache_lock, std::lock_guard<std::mutex> & cache_lock,
std::lock_guard<std::mutex> & segment_lock); std::unique_lock<std::mutex> & segment_lock);
void reduceSizeToDownloaded( void reduceSizeToDownloaded(
const Key & key, const Key & key,
size_t offset, size_t offset,
std::lock_guard<std::mutex> & cache_lock, std::lock_guard<std::mutex> & cache_lock,
std::lock_guard<std::mutex> & segment_lock); std::unique_lock<std::mutex> & segment_lock);
void assertInitialized(std::lock_guard<std::mutex> & cache_lock) const;
struct FileSegmentCell : private boost::noncopyable struct FileSegmentCell : private boost::noncopyable
{ {
@ -225,7 +223,7 @@ private:
size_t offset, size_t offset,
size_t size, size_t size,
FileSegment::State state, FileSegment::State state,
bool is_persistent, const CreateFileSegmentSettings & create_settings,
std::lock_guard<std::mutex> & cache_lock); std::lock_guard<std::mutex> & cache_lock);
static void useCell(const FileSegmentCell & cell, FileSegments & result, std::lock_guard<std::mutex> & cache_lock); static void useCell(const FileSegmentCell & cell, FileSegments & result, std::lock_guard<std::mutex> & cache_lock);
@ -242,7 +240,7 @@ private:
size_t offset, size_t offset,
size_t size, size_t size,
FileSegment::State state, FileSegment::State state,
bool is_persistent, const CreateFileSegmentSettings & create_settings,
std::lock_guard<std::mutex> & cache_lock); std::lock_guard<std::mutex> & cache_lock);
String dumpStructureUnlocked(const Key & key_, std::lock_guard<std::mutex> & cache_lock); String dumpStructureUnlocked(const Key & key_, std::lock_guard<std::mutex> & cache_lock);
@ -252,7 +250,7 @@ private:
const Key & key, const Key & key,
const FileSegment::Range & range, const FileSegment::Range & range,
bool fill_with_detached_file_segments, bool fill_with_detached_file_segments,
bool is_persistent, const CreateFileSegmentSettings & settings,
std::lock_guard<std::mutex> & cache_lock); std::lock_guard<std::mutex> & cache_lock);
size_t getUsedCacheSizeUnlocked(std::lock_guard<std::mutex> & cache_lock) const; size_t getUsedCacheSizeUnlocked(std::lock_guard<std::mutex> & cache_lock) const;

View File

@ -12,5 +12,6 @@ class FileCache;
using FileCachePtr = std::shared_ptr<FileCache>; using FileCachePtr = std::shared_ptr<FileCache>;
struct FileCacheSettings; struct FileCacheSettings;
struct CreateFileSegmentSettings;
} }

View File

@ -1,10 +1,10 @@
#include "FileSegment.h" #include "FileSegment.h"
#include <base/getThreadId.h> #include <base/getThreadId.h>
#include <base/scope_guard.h> #include <Common/scope_guard_safe.h>
#include <Common/hex.h>
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
#include <Interpreters/Cache/FileCache.h> #include <Interpreters/Cache/FileCache.h>
#include <Common/hex.h>
#include <IO/WriteBufferFromString.h> #include <IO/WriteBufferFromString.h>
#include <IO/Operators.h> #include <IO/Operators.h>
#include <filesystem> #include <filesystem>
@ -29,7 +29,7 @@ FileSegment::FileSegment(
const Key & key_, const Key & key_,
FileCache * cache_, FileCache * cache_,
State download_state_, State download_state_,
bool is_persistent_) const CreateFileSegmentSettings & settings)
: segment_range(offset_, offset_ + size_ - 1) : segment_range(offset_, offset_ + size_ - 1)
, download_state(download_state_) , download_state(download_state_)
, file_key(key_) , file_key(key_)
@ -39,7 +39,7 @@ FileSegment::FileSegment(
#else #else
, log(&Poco::Logger::get("FileSegment")) , log(&Poco::Logger::get("FileSegment"))
#endif #endif
, is_persistent(is_persistent_) , is_persistent(settings.is_persistent)
{ {
/// On creation, file segment state can be EMPTY, DOWNLOADED, DOWNLOADING. /// On creation, file segment state can be EMPTY, DOWNLOADED, DOWNLOADING.
switch (download_state) switch (download_state)
@ -64,50 +64,79 @@ FileSegment::FileSegment(
} }
default: default:
{ {
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Can create cell with either EMPTY, DOWNLOADED, DOWNLOADING state"); throw Exception(
ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR,
"Can create cell with either EMPTY, DOWNLOADED, DOWNLOADING state");
} }
} }
} }
String FileSegment::getPathInLocalCache() const
{
return cache->getPathInLocalCache(key(), offset(), isPersistent());
}
FileSegment::State FileSegment::state() const FileSegment::State FileSegment::state() const
{ {
std::lock_guard segment_lock(mutex); std::unique_lock segment_lock(mutex);
return download_state; return download_state;
} }
size_t FileSegment::getDownloadOffset() const void FileSegment::setDownloadState(State state)
{
LOG_TEST(log, "Updated state from {} to {}", stateToString(download_state), stateToString(state));
download_state = state;
}
size_t FileSegment::getFirstNonDownloadedOffset() const
{
std::unique_lock segment_lock(mutex);
return getFirstNonDownloadedOffsetUnlocked(segment_lock);
}
size_t FileSegment::getFirstNonDownloadedOffsetUnlocked(std::unique_lock<std::mutex> & segment_lock) const
{ {
std::lock_guard segment_lock(mutex);
return range().left + getDownloadedSizeUnlocked(segment_lock); return range().left + getDownloadedSizeUnlocked(segment_lock);
} }
size_t FileSegment::getCurrentWriteOffset() const
{
std::unique_lock segment_lock(mutex);
return getCurrentWriteOffsetUnlocked(segment_lock);
}
size_t FileSegment::getCurrentWriteOffsetUnlocked(std::unique_lock<std::mutex> & segment_lock) const
{
return getFirstNonDownloadedOffsetUnlocked(segment_lock);
}
size_t FileSegment::getDownloadedSize() const size_t FileSegment::getDownloadedSize() const
{ {
std::lock_guard segment_lock(mutex); std::unique_lock segment_lock(mutex);
return getDownloadedSizeUnlocked(segment_lock); return getDownloadedSizeUnlocked(segment_lock);
} }
size_t FileSegment::getRemainingSizeToDownload() const size_t FileSegment::getDownloadedSizeUnlocked(std::unique_lock<std::mutex> & /* segment_lock */) const
{
std::lock_guard segment_lock(mutex);
return range().size() - getDownloadedSizeUnlocked(segment_lock);
}
bool FileSegment::isDetached() const
{
std::lock_guard segment_lock(mutex);
return is_detached;
}
size_t FileSegment::getDownloadedSizeUnlocked(std::lock_guard<std::mutex> & /* segment_lock */) const
{ {
if (download_state == State::DOWNLOADED) if (download_state == State::DOWNLOADED)
return downloaded_size; return downloaded_size;
std::lock_guard download_lock(download_mutex); std::unique_lock download_lock(download_mutex);
return downloaded_size; return downloaded_size;
} }
size_t FileSegment::getRemainingSizeToDownload() const
{
std::unique_lock segment_lock(mutex);
return range().size() - getDownloadedSizeUnlocked(segment_lock);
}
bool FileSegment::isDownloaded() const
{
std::lock_guard segment_lock(mutex);
return is_downloaded;
}
String FileSegment::getCallerId() String FileSegment::getCallerId()
{ {
if (!CurrentThread::isInitialized() if (!CurrentThread::isInitialized()
@ -118,84 +147,106 @@ String FileSegment::getCallerId()
return std::string(CurrentThread::getQueryId()) + ":" + toString(getThreadId()); return std::string(CurrentThread::getQueryId()) + ":" + toString(getThreadId());
} }
String FileSegment::getDownloader() const
{
std::unique_lock segment_lock(mutex);
return getDownloaderUnlocked(segment_lock);
}
String FileSegment::getDownloaderUnlocked(std::unique_lock<std::mutex> & /* segment_lock */) const
{
return downloader_id;
}
String FileSegment::getOrSetDownloader() String FileSegment::getOrSetDownloader()
{ {
std::lock_guard segment_lock(mutex); std::unique_lock segment_lock(mutex);
assertNotDetached(segment_lock); assertNotDetachedUnlocked(segment_lock);
if (downloader_id.empty()) auto current_downloader = getDownloaderUnlocked(segment_lock);
if (current_downloader.empty())
{ {
assert(download_state != State::DOWNLOADING); bool allow_new_downloader = download_state == State::EMPTY || download_state == State::PARTIALLY_DOWNLOADED;
if (!allow_new_downloader)
return "notAllowed:" + stateToString(download_state);
if (download_state != State::EMPTY current_downloader = downloader_id = getCallerId();
&& download_state != State::PARTIALLY_DOWNLOADED) setDownloadState(State::DOWNLOADING);
return "None";
downloader_id = getCallerId();
download_state = State::DOWNLOADING;
} }
else if (downloader_id == getCallerId())
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR,
"Attempt to set the same downloader for segment {} for the second time", range().toString());
return downloader_id; return current_downloader;
}
void FileSegment::resetDownloadingStateUnlocked([[maybe_unused]] std::unique_lock<std::mutex> & segment_lock)
{
assert(isDownloaderUnlocked(segment_lock));
assert(download_state == State::DOWNLOADING);
size_t current_downloaded_size = getDownloadedSizeUnlocked(segment_lock);
/// range().size() can equal 0 in case of write-though cache.
if (current_downloaded_size != 0 && current_downloaded_size == range().size())
setDownloadedUnlocked(segment_lock);
else
setDownloadState(State::PARTIALLY_DOWNLOADED);
} }
void FileSegment::resetDownloader() void FileSegment::resetDownloader()
{ {
std::lock_guard segment_lock(mutex); std::unique_lock segment_lock(mutex);
assertNotDetached(segment_lock); assertNotDetachedUnlocked(segment_lock);
assertIsDownloaderUnlocked("resetDownloader", segment_lock);
if (downloader_id.empty()) resetDownloadingStateUnlocked(segment_lock);
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "There is no downloader"); resetDownloaderUnlocked(segment_lock);
if (getCallerId() != downloader_id)
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Downloader can be reset only by downloader");
resetDownloaderImpl(segment_lock);
} }
void FileSegment::resetDownloaderImpl(std::lock_guard<std::mutex> & segment_lock) void FileSegment::resetDownloaderUnlocked(std::unique_lock<std::mutex> & /* segment_lock */)
{ {
if (getDownloadedSizeUnlocked(segment_lock) == range().size()) LOG_TEST(log, "Resetting downloader from {}", downloader_id);
setDownloaded(segment_lock);
else
download_state = State::PARTIALLY_DOWNLOADED;
downloader_id.clear(); downloader_id.clear();
} }
String FileSegment::getDownloader() const void FileSegment::assertIsDownloaderUnlocked(const std::string & operation, std::unique_lock<std::mutex> & segment_lock) const
{ {
std::lock_guard segment_lock(mutex); auto caller = getCallerId();
return downloader_id; auto current_downloader = getDownloaderUnlocked(segment_lock);
LOG_TEST(log, "Downloader id: {}, caller id: {}", current_downloader, caller);
if (caller != current_downloader)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Operation `{}` can be done only by downloader. "
"(CallerId: {}, downloader id: {})",
operation, caller, downloader_id);
}
} }
bool FileSegment::isDownloader() const bool FileSegment::isDownloader() const
{ {
std::lock_guard segment_lock(mutex); std::unique_lock segment_lock(mutex);
return getCallerId() == downloader_id; return isDownloaderUnlocked(segment_lock);
} }
bool FileSegment::isDownloaderImpl(std::lock_guard<std::mutex> & /* segment_lock */) const bool FileSegment::isDownloaderUnlocked(std::unique_lock<std::mutex> & segment_lock) const
{ {
return getCallerId() == downloader_id; return getCallerId() == getDownloaderUnlocked(segment_lock);
} }
FileSegment::RemoteFileReaderPtr FileSegment::getRemoteFileReader() FileSegment::RemoteFileReaderPtr FileSegment::getRemoteFileReader()
{ {
if (!isDownloader()) std::unique_lock segment_lock(mutex);
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Only downloader can use remote filesystem file reader"); assertIsDownloaderUnlocked("getRemoteFileReader", segment_lock);
return remote_file_reader; return remote_file_reader;
} }
FileSegment::RemoteFileReaderPtr FileSegment::extractRemoteFileReader() FileSegment::RemoteFileReaderPtr FileSegment::extractRemoteFileReader()
{ {
std::lock_guard cache_lock(cache->mutex); std::lock_guard cache_lock(cache->mutex);
std::lock_guard segment_lock(mutex); std::unique_lock segment_lock(mutex);
if (!is_detached) if (!is_detached)
{ {
@ -210,8 +261,8 @@ FileSegment::RemoteFileReaderPtr FileSegment::extractRemoteFileReader()
void FileSegment::setRemoteFileReader(RemoteFileReaderPtr remote_file_reader_) void FileSegment::setRemoteFileReader(RemoteFileReaderPtr remote_file_reader_)
{ {
if (!isDownloader()) std::unique_lock segment_lock(mutex);
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Only downloader can use remote filesystem file reader"); assertIsDownloaderUnlocked("setRemoteFileReader", segment_lock);
if (remote_file_reader) if (remote_file_reader)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Remote file reader already exists"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Remote file reader already exists");
@ -221,8 +272,8 @@ void FileSegment::setRemoteFileReader(RemoteFileReaderPtr remote_file_reader_)
void FileSegment::resetRemoteFileReader() void FileSegment::resetRemoteFileReader()
{ {
if (!isDownloader()) std::unique_lock segment_lock(mutex);
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Only downloader can use remote filesystem file reader"); assertIsDownloaderUnlocked("resetRemoteFileReader", segment_lock);
if (!remote_file_reader) if (!remote_file_reader)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Remote file reader does not exist"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Remote file reader does not exist");
@ -230,55 +281,59 @@ void FileSegment::resetRemoteFileReader()
remote_file_reader.reset(); remote_file_reader.reset();
} }
void FileSegment::write(const char * from, size_t size, size_t offset_) void FileSegment::write(const char * from, size_t size, size_t offset)
{ {
if (!size) if (!size)
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Writing zero size is not allowed"); throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Writing zero size is not allowed");
if (availableSize() < size) {
throw Exception( std::unique_lock segment_lock(mutex);
ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR,
"Not enough space is reserved. Available: {}, expected: {}", availableSize(), size);
if (!isDownloader()) assertIsDownloaderUnlocked("write", segment_lock);
assertNotDetachedUnlocked(segment_lock);
if (download_state != State::DOWNLOADING)
throw Exception( throw Exception(
ErrorCodes::LOGICAL_ERROR, ErrorCodes::LOGICAL_ERROR,
"Only downloader can do the downloading. (CallerId: {}, DownloaderId: {})", "Expected DOWNLOADING state, got {}", stateToString(download_state));
getCallerId(), downloader_id);
if (getDownloadedSize() == range().size()) size_t first_non_downloaded_offset = getFirstNonDownloadedOffsetUnlocked(segment_lock);
if (offset != first_non_downloaded_offset)
throw Exception( throw Exception(
ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR,
"Attempt to write {} bytes to offset: {}, but current file segment is already fully downloaded", "Attempt to write {} bytes to offset: {}, but current write offset is {}",
size, offset_); size, offset, first_non_downloaded_offset);
auto download_offset = range().left + downloaded_size; size_t current_downloaded_size = getDownloadedSizeUnlocked(segment_lock);
if (offset_ != download_offset) chassert(reserved_size >= current_downloaded_size);
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, size_t free_reserved_size = reserved_size - current_downloaded_size;
"Attempt to write {} bytes to offset: {}, but current download offset is {}",
size, offset_, download_offset);
{ if (free_reserved_size < size)
std::lock_guard segment_lock(mutex); throw Exception(
assertNotDetached(segment_lock); ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR,
} "Not enough space is reserved. Available: {}, expected: {}", free_reserved_size, size);
if (current_downloaded_size == range().size())
throw Exception(ErrorCodes::LOGICAL_ERROR, "File segment is already fully downloaded");
if (!cache_writer) if (!cache_writer)
{ {
if (downloaded_size > 0) if (current_downloaded_size > 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Cache writer was finalized (downloaded size: {}, state: {})", "Cache writer was finalized (downloaded size: {}, state: {})",
downloaded_size, stateToString(download_state)); current_downloaded_size, stateToString(download_state));
auto download_path = getPathInLocalCache(); auto download_path = getPathInLocalCache();
cache_writer = std::make_unique<WriteBufferFromFile>(download_path); cache_writer = std::make_unique<WriteBufferFromFile>(download_path);
} }
}
try try
{ {
cache_writer->write(from, size); cache_writer->write(from, size);
std::lock_guard download_lock(download_mutex); std::unique_lock download_lock(download_mutex);
cache_writer->next(); cache_writer->next();
@ -286,23 +341,20 @@ void FileSegment::write(const char * from, size_t size, size_t offset_)
} }
catch (Exception & e) catch (Exception & e)
{ {
std::lock_guard segment_lock(mutex); std::unique_lock segment_lock(mutex);
wrapWithCacheInfo(e, "while writing into cache", segment_lock); wrapWithCacheInfo(e, "while writing into cache", segment_lock);
setDownloadFailed(segment_lock); setDownloadFailedUnlocked(segment_lock);
cv.notify_all(); cv.notify_all();
throw; throw;
} }
assert(getDownloadOffset() == offset_ + size); #ifndef NDEBUG
} chassert(getFirstNonDownloadedOffset() == offset + size);
#endif
String FileSegment::getPathInLocalCache() const
{
return cache->getPathInLocalCache(key(), offset(), isPersistent());
} }
FileSegment::State FileSegment::wait() FileSegment::State FileSegment::wait()
@ -324,8 +376,8 @@ FileSegment::State FileSegment::wait()
{ {
LOG_TEST(log, "{} waiting on: {}, current downloader: {}", getCallerId(), range().toString(), downloader_id); LOG_TEST(log, "{} waiting on: {}, current downloader: {}", getCallerId(), range().toString(), downloader_id);
assert(!downloader_id.empty()); chassert(!getDownloaderUnlocked(segment_lock).empty());
assert(downloader_id != getCallerId()); chassert(!isDownloaderUnlocked(segment_lock));
cv.wait_for(segment_lock, std::chrono::seconds(60)); cv.wait_for(segment_lock, std::chrono::seconds(60));
} }
@ -338,30 +390,23 @@ bool FileSegment::reserve(size_t size_to_reserve)
if (!size_to_reserve) if (!size_to_reserve)
throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Zero space reservation is not allowed"); throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Zero space reservation is not allowed");
{ size_t expected_downloaded_size;
std::lock_guard segment_lock(mutex);
assertNotDetached(segment_lock);
auto caller_id = getCallerId();
bool is_downloader = caller_id == downloader_id;
if (!is_downloader)
{ {
throw Exception( std::unique_lock segment_lock(mutex);
ErrorCodes::LOGICAL_ERROR,
"Space can be reserved only by downloader (current: {}, expected: {})",
caller_id, downloader_id);
}
size_t current_downloaded_size = getDownloadedSizeUnlocked(segment_lock); assertNotDetachedUnlocked(segment_lock);
if (current_downloaded_size + size_to_reserve > range().size()) assertIsDownloaderUnlocked("reserve", segment_lock);
{
expected_downloaded_size = getDownloadedSizeUnlocked(segment_lock);
if (expected_downloaded_size + size_to_reserve > range().size())
throw Exception( throw Exception(
ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR,
"Attempt to reserve space too much space: {} ({})", "Attempt to reserve space too much space ({}) for file segment with range: {} (downloaded size: {})",
size_to_reserve, getInfoForLogImpl(segment_lock)); size_to_reserve, range().toString(), downloaded_size);
}
assert(reserved_size >= current_downloaded_size); chassert(reserved_size >= expected_downloaded_size);
} }
/** /**
@ -370,9 +415,7 @@ bool FileSegment::reserve(size_t size_to_reserve)
* and the caller is going to continue; * and the caller is going to continue;
*/ */
size_t current_downloaded_size = getDownloadedSize(); size_t already_reserved_size = reserved_size - expected_downloaded_size;
assert(reserved_size >= current_downloaded_size);
size_t already_reserved_size = reserved_size - current_downloaded_size;
bool reserved = already_reserved_size >= size_to_reserve; bool reserved = already_reserved_size >= size_to_reserve;
if (!reserved) if (!reserved)
@ -392,23 +435,13 @@ bool FileSegment::reserve(size_t size_to_reserve)
return reserved; return reserved;
} }
bool FileSegment::isDownloaded() const void FileSegment::setDownloadedUnlocked([[maybe_unused]] std::unique_lock<std::mutex> & segment_lock)
{
std::lock_guard segment_lock(mutex);
return isDownloadedUnlocked(segment_lock);
}
bool FileSegment::isDownloadedUnlocked(std::lock_guard<std::mutex> & /* segment_lock */) const
{
return is_downloaded;
}
void FileSegment::setDownloaded([[maybe_unused]] std::lock_guard<std::mutex> & segment_lock)
{ {
if (is_downloaded) if (is_downloaded)
return; return;
downloader_id.clear(); setDownloadState(State::DOWNLOADED);
is_downloaded = true;
if (cache_writer) if (cache_writer)
{ {
@ -424,10 +457,12 @@ void FileSegment::setDownloaded([[maybe_unused]] std::lock_guard<std::mutex> & s
assert(std::filesystem::file_size(getPathInLocalCache()) > 0); assert(std::filesystem::file_size(getPathInLocalCache()) > 0);
} }
void FileSegment::setDownloadFailed(std::lock_guard<std::mutex> & /* segment_lock */) void FileSegment::setDownloadFailedUnlocked(std::unique_lock<std::mutex> & segment_lock)
{ {
download_state = State::PARTIALLY_DOWNLOADED_NO_CONTINUATION; LOG_INFO(log, "Settings download as failed: {}", getInfoForLogUnlocked(segment_lock));
downloader_id.clear();
setDownloadState(State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
resetDownloaderUnlocked(segment_lock);
if (cache_writer) if (cache_writer)
{ {
@ -437,43 +472,31 @@ void FileSegment::setDownloadFailed(std::lock_guard<std::mutex> & /* segment_loc
} }
} }
void FileSegment::completeBatchAndResetDownloader() void FileSegment::completePartAndResetDownloader()
{ {
std::lock_guard segment_lock(mutex); std::unique_lock segment_lock(mutex);
completePartAndResetDownloaderUnlocked(segment_lock);
}
assertNotDetached(segment_lock); void FileSegment::completePartAndResetDownloaderUnlocked(std::unique_lock<std::mutex> & segment_lock)
{
assertNotDetachedUnlocked(segment_lock);
assertIsDownloaderUnlocked("completePartAndResetDownloader", segment_lock);
if (!isDownloaderImpl(segment_lock)) resetDownloadingStateUnlocked(segment_lock);
{ resetDownloaderUnlocked(segment_lock);
cv.notify_all();
throw Exception(
ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR,
"File segment can be completed only by downloader ({} != {})",
downloader_id, getCallerId());
}
resetDownloaderImpl(segment_lock);
LOG_TEST(log, "Complete batch. Current downloaded size: {}", getDownloadedSizeUnlocked(segment_lock));
LOG_TEST(log, "Complete batch. ({})", getInfoForLogUnlocked(segment_lock));
cv.notify_all(); cv.notify_all();
} }
void FileSegment::completeWithState(State state) void FileSegment::completeWithState(State state)
{ {
std::lock_guard cache_lock(cache->mutex); std::lock_guard cache_lock(cache->mutex);
std::lock_guard segment_lock(mutex); std::unique_lock segment_lock(mutex);
assertNotDetached(segment_lock); assertNotDetachedUnlocked(segment_lock);
assertIsDownloaderUnlocked("complete", segment_lock);
auto caller_id = getCallerId();
if (caller_id != downloader_id)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"File segment completion can be done only by downloader. (CallerId: {}, downloader id: {}",
caller_id, downloader_id);
}
if (state != State::DOWNLOADED if (state != State::DOWNLOADED
&& state != State::PARTIALLY_DOWNLOADED && state != State::PARTIALLY_DOWNLOADED
@ -485,24 +508,29 @@ void FileSegment::completeWithState(State state)
"Cannot complete file segment with state: {}", stateToString(state)); "Cannot complete file segment with state: {}", stateToString(state));
} }
download_state = state; setDownloadState(state);
completeBasedOnCurrentState(cache_lock, segment_lock); completeBasedOnCurrentState(cache_lock, segment_lock);
} }
void FileSegment::completeWithoutState(std::lock_guard<std::mutex> & cache_lock) void FileSegment::completeWithoutState()
{ {
std::lock_guard segment_lock(mutex); std::lock_guard cache_lock(cache->mutex);
completeWithoutStateUnlocked(cache_lock);
}
void FileSegment::completeWithoutStateUnlocked(std::lock_guard<std::mutex> & cache_lock)
{
std::unique_lock segment_lock(mutex);
completeBasedOnCurrentState(cache_lock, segment_lock); completeBasedOnCurrentState(cache_lock, segment_lock);
} }
void FileSegment::completeBasedOnCurrentState(std::lock_guard<std::mutex> & cache_lock, std::lock_guard<std::mutex> & segment_lock) void FileSegment::completeBasedOnCurrentState(std::lock_guard<std::mutex> & cache_lock, std::unique_lock<std::mutex> & segment_lock)
{ {
if (is_detached) if (is_detached)
return; return;
bool is_downloader = isDownloaderImpl(segment_lock); bool is_downloader = isDownloaderUnlocked(segment_lock);
bool is_last_holder = cache->isLastFileSegmentHolder(key(), offset(), cache_lock, segment_lock); bool is_last_holder = cache->isLastFileSegmentHolder(key(), offset(), cache_lock, segment_lock);
bool can_update_segment_state = is_downloader || is_last_holder;
size_t current_downloaded_size = getDownloadedSizeUnlocked(segment_lock); size_t current_downloaded_size = getDownloadedSizeUnlocked(segment_lock);
SCOPE_EXIT({ SCOPE_EXIT({
@ -512,16 +540,16 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard<std::mutex> & cach
} }
}); });
LOG_TEST(log, "Complete without state (is_last_holder: {}). File segment info: {}", is_last_holder, getInfoForLogImpl(segment_lock)); LOG_TEST(
log,
"Complete based on current state (is_last_holder: {}, {})",
is_last_holder, getInfoForLogUnlocked(segment_lock));
if (can_update_segment_state) if (is_downloader)
{ {
if (current_downloaded_size == range().size()) if (download_state == State::DOWNLOADING) /// != in case of completeWithState
setDownloaded(segment_lock); resetDownloadingStateUnlocked(segment_lock);
else resetDownloaderUnlocked(segment_lock);
download_state = State::PARTIALLY_DOWNLOADED;
resetDownloaderImpl(segment_lock);
} }
switch (download_state) switch (download_state)
@ -535,16 +563,17 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard<std::mutex> & cach
} }
case State::DOWNLOADED: case State::DOWNLOADED:
{ {
assert(getDownloadedSizeUnlocked(segment_lock) == range().size()); chassert(getDownloadedSizeUnlocked(segment_lock) == range().size());
assert(isDownloadedUnlocked(segment_lock)); assert(is_downloaded);
assert(!cache_writer);
break; break;
} }
case State::DOWNLOADING: case State::DOWNLOADING:
case State::EMPTY:
{ {
assert(!is_last_holder); chassert(!is_last_holder);
break; break;
} }
case State::EMPTY:
case State::PARTIALLY_DOWNLOADED: case State::PARTIALLY_DOWNLOADED:
case State::PARTIALLY_DOWNLOADED_NO_CONTINUATION: case State::PARTIALLY_DOWNLOADED_NO_CONTINUATION:
{ {
@ -554,7 +583,7 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard<std::mutex> & cach
{ {
LOG_TEST(log, "Remove cell {} (nothing downloaded)", range().toString()); LOG_TEST(log, "Remove cell {} (nothing downloaded)", range().toString());
download_state = State::SKIP_CACHE; setDownloadState(State::SKIP_CACHE);
cache->remove(key(), offset(), cache_lock, segment_lock); cache->remove(key(), offset(), cache_lock, segment_lock);
} }
else else
@ -567,7 +596,7 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard<std::mutex> & cach
* in FileSegmentsHolder represent a contiguous range, so we can resize * in FileSegmentsHolder represent a contiguous range, so we can resize
* it only when nobody needs it. * it only when nobody needs it.
*/ */
download_state = State::PARTIALLY_DOWNLOADED_NO_CONTINUATION; setDownloadState(State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
/// Resize this file segment by creating a copy file segment with DOWNLOADED state, /// Resize this file segment by creating a copy file segment with DOWNLOADED state,
/// but current file segment should remain PARRTIALLY_DOWNLOADED_NO_CONTINUATION and with detached state, /// but current file segment should remain PARRTIALLY_DOWNLOADED_NO_CONTINUATION and with detached state,
@ -576,23 +605,22 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard<std::mutex> & cach
cache->reduceSizeToDownloaded(key(), offset(), cache_lock, segment_lock); cache->reduceSizeToDownloaded(key(), offset(), cache_lock, segment_lock);
} }
markAsDetached(segment_lock); detachAssumeStateFinalized(segment_lock);
} }
break; break;
} }
} }
LOG_TEST(log, "Completed file segment: {}", getInfoForLogImpl(segment_lock)); LOG_TEST(log, "Completed file segment: {}", getInfoForLogUnlocked(segment_lock));
assertCorrectnessImpl(segment_lock);
} }
String FileSegment::getInfoForLog() const String FileSegment::getInfoForLog() const
{ {
std::lock_guard segment_lock(mutex); std::unique_lock segment_lock(mutex);
return getInfoForLogImpl(segment_lock); return getInfoForLogUnlocked(segment_lock);
} }
String FileSegment::getInfoForLogImpl(std::lock_guard<std::mutex> & segment_lock) const String FileSegment::getInfoForLogUnlocked(std::unique_lock<std::mutex> & segment_lock) const
{ {
WriteBufferFromOwnString info; WriteBufferFromOwnString info;
info << "File segment: " << range().toString() << ", "; info << "File segment: " << range().toString() << ", ";
@ -601,15 +629,18 @@ String FileSegment::getInfoForLogImpl(std::lock_guard<std::mutex> & segment_lock
info << "downloaded size: " << getDownloadedSizeUnlocked(segment_lock) << ", "; info << "downloaded size: " << getDownloadedSizeUnlocked(segment_lock) << ", ";
info << "reserved size: " << reserved_size << ", "; info << "reserved size: " << reserved_size << ", ";
info << "downloader id: " << (downloader_id.empty() ? "None" : downloader_id) << ", "; info << "downloader id: " << (downloader_id.empty() ? "None" : downloader_id) << ", ";
info << "current write offset: " << getCurrentWriteOffsetUnlocked(segment_lock) << ", ";
info << "first non-downloaded offset: " << getFirstNonDownloadedOffsetUnlocked(segment_lock) << ", ";
info << "caller id: " << getCallerId() << ", "; info << "caller id: " << getCallerId() << ", ";
info << "detached: " << is_detached << ", ";
info << "persistent: " << is_persistent; info << "persistent: " << is_persistent;
return info.str(); return info.str();
} }
void FileSegment::wrapWithCacheInfo(Exception & e, const String & message, std::lock_guard<std::mutex> & segment_lock) const void FileSegment::wrapWithCacheInfo(Exception & e, const String & message, std::unique_lock<std::mutex> & segment_lock) const
{ {
e.addMessage(fmt::format("{}, current cache state: {}", message, getInfoForLogImpl(segment_lock))); e.addMessage(fmt::format("{}, current cache state: {}", message, getInfoForLogUnlocked(segment_lock)));
} }
String FileSegment::stateToString(FileSegment::State state) String FileSegment::stateToString(FileSegment::State state)
@ -634,63 +665,64 @@ String FileSegment::stateToString(FileSegment::State state)
void FileSegment::assertCorrectness() const void FileSegment::assertCorrectness() const
{ {
std::lock_guard segment_lock(mutex); std::unique_lock segment_lock(mutex);
assertCorrectnessImpl(segment_lock); assertCorrectnessUnlocked(segment_lock);
} }
void FileSegment::assertCorrectnessImpl(std::lock_guard<std::mutex> & /* segment_lock */) const void FileSegment::assertCorrectnessUnlocked(std::unique_lock<std::mutex> & segment_lock) const
{ {
assert(downloader_id.empty() == (download_state != FileSegment::State::DOWNLOADING)); auto current_downloader = getDownloaderUnlocked(segment_lock);
assert(!downloader_id.empty() == (download_state == FileSegment::State::DOWNLOADING)); chassert(current_downloader.empty() == (download_state != FileSegment::State::DOWNLOADING));
assert(download_state != FileSegment::State::DOWNLOADED || std::filesystem::file_size(getPathInLocalCache()) > 0); chassert(!current_downloader.empty() == (download_state == FileSegment::State::DOWNLOADING));
chassert(download_state != FileSegment::State::DOWNLOADED || std::filesystem::file_size(getPathInLocalCache()) > 0);
} }
void FileSegment::throwIfDetached() const void FileSegment::throwIfDetachedUnlocked(std::unique_lock<std::mutex> & segment_lock) const
{
std::lock_guard segment_lock(mutex);
throwIfDetachedUnlocked(segment_lock);
}
void FileSegment::throwIfDetachedUnlocked(std::lock_guard<std::mutex> & segment_lock) const
{ {
throw Exception( throw Exception(
ErrorCodes::LOGICAL_ERROR, ErrorCodes::LOGICAL_ERROR,
"Cache file segment is in detached state, operation not allowed. " "Cache file segment is in detached state, operation not allowed. "
"It can happen when cache was concurrently dropped with SYSTEM DROP FILESYSTEM CACHE FORCE. " "It can happen when cache was concurrently dropped with SYSTEM DROP FILESYSTEM CACHE FORCE. "
"Please, retry. File segment info: {}", getInfoForLogImpl(segment_lock)); "Please, retry. File segment info: {}", getInfoForLogUnlocked(segment_lock));
} }
void FileSegment::assertNotDetached() const
{
std::unique_lock segment_lock(mutex);
assertNotDetachedUnlocked(segment_lock);
}
void FileSegment::assertNotDetached(std::lock_guard<std::mutex> & segment_lock) const void FileSegment::assertNotDetachedUnlocked(std::unique_lock<std::mutex> & segment_lock) const
{ {
if (is_detached) if (is_detached)
throwIfDetachedUnlocked(segment_lock); throwIfDetachedUnlocked(segment_lock);
} }
void FileSegment::assertDetachedStatus(std::lock_guard<std::mutex> & segment_lock) const void FileSegment::assertDetachedStatus(std::unique_lock<std::mutex> & segment_lock) const
{ {
/// Detached file segment is allowed to have only a certain subset of states. /// Detached file segment is allowed to have only a certain subset of states.
/// It should be either EMPTY or one of the finalized states. /// It should be either EMPTY or one of the finalized states.
if (download_state != State::EMPTY && !hasFinalizedState()) if (download_state != State::EMPTY && !hasFinalizedStateUnlocked(segment_lock))
{ {
throw Exception( throw Exception(
ErrorCodes::LOGICAL_ERROR, ErrorCodes::LOGICAL_ERROR,
"Detached file segment has incorrect state: {}", "Detached file segment has incorrect state: {}",
getInfoForLogImpl(segment_lock)); getInfoForLogUnlocked(segment_lock));
} }
} }
FileSegmentPtr FileSegment::getSnapshot(const FileSegmentPtr & file_segment, std::lock_guard<std::mutex> & /* cache_lock */) FileSegmentPtr FileSegment::getSnapshot(const FileSegmentPtr & file_segment, std::lock_guard<std::mutex> & /* cache_lock */)
{ {
std::lock_guard segment_lock(file_segment->mutex); std::unique_lock segment_lock(file_segment->mutex);
auto snapshot = std::make_shared<FileSegment>( auto snapshot = std::make_shared<FileSegment>(
file_segment->offset(), file_segment->offset(),
file_segment->range().size(), file_segment->range().size(),
file_segment->key(), file_segment->key(),
nullptr, nullptr,
State::EMPTY); State::EMPTY,
CreateFileSegmentSettings{});
snapshot->hits_count = file_segment->getHitsCount(); snapshot->hits_count = file_segment->getHitsCount();
snapshot->ref_count = file_segment.use_count(); snapshot->ref_count = file_segment.use_count();
@ -701,41 +733,43 @@ FileSegmentPtr FileSegment::getSnapshot(const FileSegmentPtr & file_segment, std
return snapshot; return snapshot;
} }
bool FileSegment::hasFinalizedState() const bool FileSegment::hasFinalizedStateUnlocked(std::unique_lock<std::mutex> & /* segment_lock */) const
{ {
return download_state == State::DOWNLOADED return download_state == State::DOWNLOADED
|| download_state == State::PARTIALLY_DOWNLOADED_NO_CONTINUATION || download_state == State::PARTIALLY_DOWNLOADED_NO_CONTINUATION
|| download_state == State::SKIP_CACHE; || download_state == State::SKIP_CACHE;
} }
void FileSegment::detach( bool FileSegment::isDetached() const
std::lock_guard<std::mutex> & /* cache_lock */, {
std::lock_guard<std::mutex> & segment_lock) std::unique_lock segment_lock(mutex);
return is_detached;
}
void FileSegment::detach(std::lock_guard<std::mutex> & /* cache_lock */, std::unique_lock<std::mutex> & segment_lock)
{ {
/// Now detached status can be in 2 cases, which do not do any complex logic:
/// 1. there is only 1 remaining file segment holder
/// && it does not need this segment anymore
/// && this file segment was in cache and needs to be removed
/// 2. in read_from_cache_if_exists_otherwise_bypass_cache case
if (is_detached) if (is_detached)
return; return;
markAsDetached(segment_lock); if (download_state == State::DOWNLOADING)
download_state = State::PARTIALLY_DOWNLOADED_NO_CONTINUATION; resetDownloadingStateUnlocked(segment_lock);
downloader_id.clear(); else
setDownloadState(State::PARTIALLY_DOWNLOADED_NO_CONTINUATION);
LOG_DEBUG(log, "Detached file segment: {}", getInfoForLogImpl(segment_lock)); resetDownloaderUnlocked(segment_lock);
detachAssumeStateFinalized(segment_lock);
} }
void FileSegment::markAsDetached(std::lock_guard<std::mutex> & /* segment_lock */) void FileSegment::detachAssumeStateFinalized(std::unique_lock<std::mutex> & segment_lock)
{ {
is_detached = true; is_detached = true;
CurrentMetrics::add(CurrentMetrics::CacheDetachedFileSegments); CurrentMetrics::add(CurrentMetrics::CacheDetachedFileSegments);
LOG_TEST(log, "Detached file segment: {}", getInfoForLogUnlocked(segment_lock));
} }
FileSegment::~FileSegment() FileSegment::~FileSegment()
{ {
std::lock_guard segment_lock(mutex); std::unique_lock segment_lock(mutex);
if (is_detached) if (is_detached)
CurrentMetrics::sub(CurrentMetrics::CacheDetachedFileSegments); CurrentMetrics::sub(CurrentMetrics::CacheDetachedFileSegments);
} }
@ -761,7 +795,7 @@ FileSegmentsHolder::~FileSegmentsHolder()
bool is_detached = false; bool is_detached = false;
{ {
std::lock_guard segment_lock(file_segment->mutex); std::unique_lock segment_lock(file_segment->mutex);
is_detached = file_segment->isDetached(segment_lock); is_detached = file_segment->isDetached(segment_lock);
if (is_detached) if (is_detached)
file_segment->assertDetachedStatus(segment_lock); file_segment->assertDetachedStatus(segment_lock);
@ -779,7 +813,7 @@ FileSegmentsHolder::~FileSegmentsHolder()
/// under the same mutex, because complete() checks for segment pointers. /// under the same mutex, because complete() checks for segment pointers.
std::lock_guard cache_lock(cache->mutex); std::lock_guard cache_lock(cache->mutex);
file_segment->completeWithoutState(cache_lock); file_segment->completeWithoutStateUnlocked(cache_lock);
file_segment_it = file_segments.erase(current_file_segment_it); file_segment_it = file_segments.erase(current_file_segment_it);
} }
@ -822,10 +856,16 @@ FileSegments::iterator FileSegmentRangeWriter::allocateFileSegment(size_t offset
std::lock_guard cache_lock(cache->mutex); std::lock_guard cache_lock(cache->mutex);
CreateFileSegmentSettings create_settings
{
.is_persistent = is_persistent,
};
/// We set max_file_segment_size to be downloaded, /// We set max_file_segment_size to be downloaded,
/// if we have less size to write, file segment will be resized in complete() method. /// if we have less size to write, file segment will be resized in complete() method.
auto file_segment = cache->createFileSegmentForDownload( auto file_segment = cache->createFileSegmentForDownload(
key, offset, cache->max_file_segment_size, is_persistent, cache_lock); key, offset, cache->max_file_segment_size, create_settings, cache_lock);
return file_segments_holder.add(std::move(file_segment)); return file_segments_holder.add(std::move(file_segment));
} }
@ -850,19 +890,18 @@ void FileSegmentRangeWriter::completeFileSegment(FileSegment & file_segment)
/// and therefore cannot be concurrently accessed. Nevertheless, it can be /// and therefore cannot be concurrently accessed. Nevertheless, it can be
/// accessed by cache system tables if someone read from them, /// accessed by cache system tables if someone read from them,
/// therefore we need a mutex. /// therefore we need a mutex.
std::lock_guard segment_lock(file_segment.mutex); std::unique_lock segment_lock(file_segment.mutex);
assert(current_downloaded_size <= file_segment.range().size()); assert(current_downloaded_size <= file_segment.range().size());
file_segment.segment_range = FileSegment::Range( file_segment.segment_range = FileSegment::Range(
file_segment.segment_range.left, file_segment.segment_range.left,
file_segment.segment_range.left + current_downloaded_size - 1); file_segment.segment_range.left + current_downloaded_size - 1);
file_segment.reserved_size = current_downloaded_size; file_segment.reserved_size = current_downloaded_size;
file_segment.setDownloadedUnlocked(segment_lock);
} }
{ file_segment.completeWithoutState();
std::lock_guard cache_lock(cache->mutex);
file_segment.completeWithoutState(cache_lock);
}
on_complete_file_segment_func(file_segment); on_complete_file_segment_func(file_segment);
} }
@ -893,18 +932,20 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset
offset, current_file_segment_write_offset); offset, current_file_segment_write_offset);
} }
size_t current_write_offset = (*current_file_segment_it)->getCurrentWriteOffset();
auto current_file_segment = *current_file_segment_it; auto current_file_segment = *current_file_segment_it;
if (current_file_segment->getRemainingSizeToDownload() == 0) if (current_file_segment->getRemainingSizeToDownload() == 0)
{ {
completeFileSegment(*current_file_segment); completeFileSegment(*current_file_segment);
current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, is_persistent); current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, is_persistent);
} }
else if (current_file_segment->getDownloadOffset() != offset) else if (current_write_offset != offset)
{ {
throw Exception( throw Exception(
ErrorCodes::LOGICAL_ERROR, ErrorCodes::LOGICAL_ERROR,
"Cannot file segment download offset {} does not match current write offset {}", "Cannot file segment download offset {} does not match current write offset {}",
current_file_segment->getDownloadOffset(), offset); current_write_offset, offset);
} }
} }
@ -915,7 +956,10 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset
throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to set a downloader. ({})", file_segment->getInfoForLog()); throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to set a downloader. ({})", file_segment->getInfoForLog());
SCOPE_EXIT({ SCOPE_EXIT({
file_segment->resetDownloader(); if (file_segment->isDownloader())
{
file_segment->completePartAndResetDownloader();
}
}); });
bool reserved = file_segment->reserve(size); bool reserved = file_segment->reserve(size);
@ -932,7 +976,17 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset
return false; return false;
} }
(*current_file_segment_it)->write(data, size, offset); try
{
file_segment->write(data, size, offset);
}
catch (...)
{
file_segment->completePartAndResetDownloader();
throw;
}
file_segment->completePartAndResetDownloader();
current_file_segment_write_offset += size; current_file_segment_write_offset += size;
return true; return true;

View File

@ -1,11 +1,15 @@
#pragma once #pragma once
#include <Core/Types.h> #include <boost/noncopyable.hpp>
#include <Interpreters/Cache/FileCacheKey.h>
#include <IO/WriteBufferFromFile.h> #include <IO/WriteBufferFromFile.h>
#include <IO/ReadBufferFromFileBase.h> #include <IO/ReadBufferFromFileBase.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <base/getThreadId.h>
#include <list> #include <list>
#include <Interpreters/Cache/FileCacheKey.h> #include <queue>
namespace Poco { class Logger; } namespace Poco { class Logger; }
@ -26,17 +30,25 @@ using FileSegmentPtr = std::shared_ptr<FileSegment>;
using FileSegments = std::list<FileSegmentPtr>; using FileSegments = std::list<FileSegmentPtr>;
class FileSegment : boost::noncopyable struct CreateFileSegmentSettings
{
bool is_persistent = false;
};
class FileSegment : private boost::noncopyable, public std::enable_shared_from_this<FileSegment>
{ {
friend class FileCache; friend class FileCache;
friend struct FileSegmentsHolder; friend struct FileSegmentsHolder;
friend class FileSegmentRangeWriter; friend class FileSegmentRangeWriter;
friend class StorageSystemFilesystemCache;
public: public:
using Key = FileCacheKey; using Key = FileCacheKey;
using RemoteFileReaderPtr = std::shared_ptr<ReadBufferFromFileBase>; using RemoteFileReaderPtr = std::shared_ptr<ReadBufferFromFileBase>;
using LocalCacheWriterPtr = std::unique_ptr<WriteBufferFromFile>; using LocalCacheWriterPtr = std::unique_ptr<WriteBufferFromFile>;
using Downloader = std::string;
using DownloaderId = std::string;
enum class State enum class State
{ {
@ -78,7 +90,7 @@ public:
const Key & key_, const Key & key_,
FileCache * cache_, FileCache * cache_,
State download_state_, State download_state_,
bool is_persistent_ = false); const CreateFileSegmentSettings & create_settings);
~FileSegment(); ~FileSegment();
@ -101,6 +113,14 @@ public:
String toString() const { return fmt::format("[{}, {}]", std::to_string(left), std::to_string(right)); } String toString() const { return fmt::format("[{}, {}]", std::to_string(left), std::to_string(right)); }
}; };
static String getCallerId();
String getInfoForLog() const;
/**
* ========== Methods to get file segment's constant state ==================
*/
const Range & range() const { return segment_range; } const Range & range() const { return segment_range; }
const Key & key() const { return file_key; } const Key & key() const { return file_key; }
@ -109,11 +129,85 @@ public:
bool isPersistent() const { return is_persistent; } bool isPersistent() const { return is_persistent; }
using UniqueId = std::pair<FileCacheKey, size_t>;
UniqueId getUniqueId() const { return std::pair(key(), offset()); }
String getPathInLocalCache() const;
/**
* ========== Methods for _any_ file segment's owner ========================
*/
String getOrSetDownloader();
bool isDownloader() const;
DownloaderId getDownloader() const;
/// Wait for the change of state from DOWNLOADING to any other.
State wait(); State wait();
bool reserve(size_t size); bool isDownloaded() const;
void write(const char * from, size_t size, size_t offset_); size_t getHitsCount() const { return hits_count; }
size_t getRefCount() const { return ref_count; }
void incrementHitsCount() { ++hits_count; }
size_t getCurrentWriteOffset() const;
size_t getFirstNonDownloadedOffset() const;
size_t getDownloadedSize() const;
/// Now detached status can be used in the following cases:
/// 1. there is only 1 remaining file segment holder
/// && it does not need this segment anymore
/// && this file segment was in cache and needs to be removed
/// 2. in read_from_cache_if_exists_otherwise_bypass_cache case to create NOOP file segments.
/// 3. removeIfExists - method which removes file segments from cache even though
/// it might be used at the moment.
/// If file segment is detached it means the following:
/// 1. It is not present in FileCache, e.g. will not be visible to any cache user apart from
/// those who acquired shared pointer to this file segment before it was detached.
/// 2. Detached file segment can still be hold by some cache users, but it's state became
/// immutable at the point it was detached, any non-const / stateful method will throw an
/// exception.
void detach(std::lock_guard<std::mutex> & cache_lock, std::unique_lock<std::mutex> & segment_lock);
static FileSegmentPtr getSnapshot(const FileSegmentPtr & file_segment, std::lock_guard<std::mutex> & cache_lock);
bool isDetached() const;
void assertCorrectness() const;
/**
* ========== Methods for _only_ file segment's `writer` ======================
*/
void synchronousWrite(const char * from, size_t size, size_t offset);
/**
* ========== Methods for _only_ file segment's `downloader` ==================
*/
/// Try to reserve exactly `size` bytes.
bool reserve(size_t size_to_reserve);
/// Write data into reserved space.
void write(const char * from, size_t size, size_t offset);
/// Complete file segment with a certain state.
void completeWithState(State state);
void completeWithoutState();
/// Complete file segment's part which was last written.
void completePartAndResetDownloader();
void resetDownloader();
RemoteFileReaderPtr getRemoteFileReader(); RemoteFileReaderPtr getRemoteFileReader();
@ -123,91 +217,55 @@ public:
void resetRemoteFileReader(); void resetRemoteFileReader();
String getOrSetDownloader();
String getDownloader() const;
void resetDownloader();
bool isDownloader() const;
bool isDownloaded() const;
static String getCallerId();
size_t getDownloadOffset() const;
size_t getDownloadedSize() const;
size_t getRemainingSizeToDownload() const; size_t getRemainingSizeToDownload() const;
void completeBatchAndResetDownloader();
void completeWithState(State state);
String getInfoForLog() const;
size_t getHitsCount() const { return hits_count; }
size_t getRefCount() const { return ref_count; }
void incrementHitsCount() { ++hits_count; }
void assertCorrectness() const;
static FileSegmentPtr getSnapshot(
const FileSegmentPtr & file_segment,
std::lock_guard<std::mutex> & cache_lock);
void detach(
std::lock_guard<std::mutex> & cache_lock,
std::lock_guard<std::mutex> & segment_lock);
[[noreturn]] void throwIfDetached() const;
bool isDetached() const;
String getPathInLocalCache() const;
private: private:
size_t availableSize() const { return reserved_size - downloaded_size; } size_t getFirstNonDownloadedOffsetUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
size_t getCurrentWriteOffsetUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
size_t getDownloadedSizeUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
size_t getDownloadedSizeUnlocked(std::lock_guard<std::mutex> & segment_lock) const; String getInfoForLogUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
String getInfoForLogImpl(std::lock_guard<std::mutex> & segment_lock) const;
void assertCorrectnessImpl(std::lock_guard<std::mutex> & segment_lock) const;
bool hasFinalizedState() const;
bool isDetached(std::lock_guard<std::mutex> & /* segment_lock */) const { return is_detached; } String getDownloaderUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
void markAsDetached(std::lock_guard<std::mutex> & segment_lock); void resetDownloaderUnlocked(std::unique_lock<std::mutex> & segment_lock);
[[noreturn]] void throwIfDetachedUnlocked(std::lock_guard<std::mutex> & segment_lock) const; void resetDownloadingStateUnlocked(std::unique_lock<std::mutex> & segment_lock);
void assertDetachedStatus(std::lock_guard<std::mutex> & segment_lock) const; void setDownloadState(State state);
void assertNotDetached(std::lock_guard<std::mutex> & segment_lock) const;
void setDownloaded(std::lock_guard<std::mutex> & segment_lock); void setDownloadedUnlocked(std::unique_lock<std::mutex> & segment_lock);
void setDownloadFailed(std::lock_guard<std::mutex> & segment_lock); void setDownloadFailedUnlocked(std::unique_lock<std::mutex> & segment_lock);
bool isDownloaderImpl(std::lock_guard<std::mutex> & segment_lock) const;
bool isDownloadedUnlocked(std::lock_guard<std::mutex> & segment_lock) const; bool hasFinalizedStateUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
void wrapWithCacheInfo(Exception & e, const String & message, std::lock_guard<std::mutex> & segment_lock) const; bool isDownloaderUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
bool lastFileSegmentHolder() const; bool isDetached(std::unique_lock<std::mutex> & /* segment_lock */) const { return is_detached; }
void detachAssumeStateFinalized(std::unique_lock<std::mutex> & segment_lock);
[[noreturn]] void throwIfDetachedUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
void assertDetachedStatus(std::unique_lock<std::mutex> & segment_lock) const;
void assertNotDetached() const;
void assertNotDetachedUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
void assertIsDownloaderUnlocked(const std::string & operation, std::unique_lock<std::mutex> & segment_lock) const;
void assertCorrectnessUnlocked(std::unique_lock<std::mutex> & segment_lock) const;
/// complete() without any completion state is called from destructor of /// complete() without any completion state is called from destructor of
/// FileSegmentsHolder. complete() might check if the caller of the method /// FileSegmentsHolder. complete() might check if the caller of the method
/// is the last alive holder of the segment. Therefore, complete() and destruction /// is the last alive holder of the segment. Therefore, complete() and destruction
/// of the file segment pointer must be done under the same cache mutex. /// of the file segment pointer must be done under the same cache mutex.
void completeBasedOnCurrentState(std::lock_guard<std::mutex> & cache_lock, std::lock_guard<std::mutex> & segment_lock); void completeWithoutStateUnlocked(std::lock_guard<std::mutex> & cache_lock);
void completeWithoutState(std::lock_guard<std::mutex> & cache_lock); void completeBasedOnCurrentState(std::lock_guard<std::mutex> & cache_lock, std::unique_lock<std::mutex> & segment_lock);
void resetDownloaderImpl(std::lock_guard<std::mutex> & segment_lock); void completePartAndResetDownloaderUnlocked(std::unique_lock<std::mutex> & segment_lock);
void wrapWithCacheInfo(Exception & e, const String & message, std::unique_lock<std::mutex> & segment_lock) const;
Range segment_range; Range segment_range;
State download_state; State download_state;
String downloader_id; /// The one who prepares the download
DownloaderId downloader_id;
RemoteFileReaderPtr remote_file_reader; RemoteFileReaderPtr remote_file_reader;
LocalCacheWriterPtr cache_writer; LocalCacheWriterPtr cache_writer;
@ -245,6 +303,7 @@ private:
std::atomic<size_t> ref_count = 0; /// Used for getting snapshot state std::atomic<size_t> ref_count = 0; /// Used for getting snapshot state
bool is_persistent; bool is_persistent;
CurrentMetrics::Increment metric_increment{CurrentMetrics::CacheFileSegments}; CurrentMetrics::Increment metric_increment{CurrentMetrics::CacheFileSegments};
}; };

View File

@ -143,7 +143,7 @@ namespace ErrorCodes
/** Set of known objects (environment), that could be used in query. /** Set of known objects (environment), that could be used in query.
* Shared (global) part. Order of members (especially, order of destruction) is very important. * Shared (global) part. Order of members (especially, order of destruction) is very important.
*/ */
struct ContextSharedPart struct ContextSharedPart : boost::noncopyable
{ {
Poco::Logger * log = &Poco::Logger::get("Context"); Poco::Logger * log = &Poco::Logger::get("Context");
@ -215,6 +215,7 @@ struct ContextSharedPart
std::unique_ptr<AccessControl> access_control; std::unique_ptr<AccessControl> access_control;
mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks. mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks.
mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files. mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files.
mutable std::unique_ptr<ThreadPool> load_marks_threadpool; /// Threadpool for loading marks cache.
mutable UncompressedCachePtr index_uncompressed_cache; /// The cache of decompressed blocks for MergeTree indices. mutable UncompressedCachePtr index_uncompressed_cache; /// The cache of decompressed blocks for MergeTree indices.
mutable MarkCachePtr index_mark_cache; /// Cache of marks in compressed files of MergeTree indices. mutable MarkCachePtr index_mark_cache; /// Cache of marks in compressed files of MergeTree indices.
mutable MMappedFileCachePtr mmap_cache; /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads. mutable MMappedFileCachePtr mmap_cache; /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads.
@ -313,11 +314,19 @@ struct ContextSharedPart
~ContextSharedPart() ~ContextSharedPart()
{ {
/// Wait for thread pool for background writes,
/// since it may use per-user MemoryTracker which will be destroyed here.
try try
{ {
/// Wait for thread pool for background writes,
/// since it may use per-user MemoryTracker which will be destroyed here.
IObjectStorage::getThreadPoolWriter().wait(); IObjectStorage::getThreadPoolWriter().wait();
/// Make sure that threadpool is destructed before this->process_list
/// because thread_status, which was created for threads inside threadpool,
/// relies on it.
if (load_marks_threadpool)
{
load_marks_threadpool->wait();
load_marks_threadpool.reset();
}
} }
catch (...) catch (...)
{ {
@ -1688,6 +1697,17 @@ void Context::dropMarkCache() const
shared->mark_cache->reset(); shared->mark_cache->reset();
} }
ThreadPool & Context::getLoadMarksThreadpool() const
{
auto lock = getLock();
if (!shared->load_marks_threadpool)
{
constexpr size_t pool_size = 50;
constexpr size_t queue_size = 1000000;
shared->load_marks_threadpool = std::make_unique<ThreadPool>(pool_size, pool_size, queue_size);
}
return *shared->load_marks_threadpool;
}
void Context::setIndexUncompressedCache(size_t max_size_in_bytes) void Context::setIndexUncompressedCache(size_t max_size_in_bytes)
{ {
@ -3429,6 +3449,8 @@ ReadSettings Context::getReadSettings() const
res.local_fs_prefetch = settings.local_filesystem_read_prefetch; res.local_fs_prefetch = settings.local_filesystem_read_prefetch;
res.remote_fs_prefetch = settings.remote_filesystem_read_prefetch; res.remote_fs_prefetch = settings.remote_filesystem_read_prefetch;
res.load_marks_asynchronously = settings.load_marks_asynchronously;
res.remote_fs_read_max_backoff_ms = settings.remote_fs_read_max_backoff_ms; res.remote_fs_read_max_backoff_ms = settings.remote_fs_read_max_backoff_ms;
res.remote_fs_read_backoff_max_tries = settings.remote_fs_read_backoff_max_tries; res.remote_fs_read_backoff_max_tries = settings.remote_fs_read_backoff_max_tries;
res.enable_filesystem_cache = settings.enable_filesystem_cache; res.enable_filesystem_cache = settings.enable_filesystem_cache;

View File

@ -806,6 +806,7 @@ public:
void setMarkCache(size_t cache_size_in_bytes, const String & mark_cache_policy); void setMarkCache(size_t cache_size_in_bytes, const String & mark_cache_policy);
std::shared_ptr<MarkCache> getMarkCache() const; std::shared_ptr<MarkCache> getMarkCache() const;
void dropMarkCache() const; void dropMarkCache() const;
ThreadPool & getLoadMarksThreadpool() const;
/// Create a cache of index uncompressed blocks of specified size. This can be done only once. /// Create a cache of index uncompressed blocks of specified size. This can be done only once.
void setIndexUncompressedCache(size_t max_size_in_bytes); void setIndexUncompressedCache(size_t max_size_in_bytes);

View File

@ -21,7 +21,7 @@ static String typeToString(FilesystemCacheLogElement::CacheType type)
case FilesystemCacheLogElement::CacheType::READ_FROM_FS_BYPASSING_CACHE: case FilesystemCacheLogElement::CacheType::READ_FROM_FS_BYPASSING_CACHE:
return "READ_FROM_FS_BYPASSING_CACHE"; return "READ_FROM_FS_BYPASSING_CACHE";
case FilesystemCacheLogElement::CacheType::WRITE_THROUGH_CACHE: case FilesystemCacheLogElement::CacheType::WRITE_THROUGH_CACHE:
return "READ_FROM_FS_BYPASSING_CACHE"; return "WRITE_THROUGH_CACHE";
} }
__builtin_unreachable(); __builtin_unreachable();
} }

View File

@ -3,9 +3,9 @@
#include <gtest/gtest.h> #include <gtest/gtest.h>
#include <Interpreters/Cache/FileCache.h> #include <Interpreters/Cache/FileCache.h>
#include <Interpreters/Cache/FileSegment.h> #include <Interpreters/Cache/FileSegment.h>
#include <Interpreters/Cache/FileCacheSettings.h>
#include <Common/CurrentThread.h> #include <Common/CurrentThread.h>
#include <Common/filesystemHelpers.h> #include <Common/filesystemHelpers.h>
#include <Interpreters/Cache/FileCacheSettings.h>
#include <Common/tests/gtest_global_context.h> #include <Common/tests/gtest_global_context.h>
#include <Common/SipHash.h> #include <Common/SipHash.h>
#include <Common/hex.h> #include <Common/hex.h>
@ -64,7 +64,7 @@ void download(DB::FileSegmentPtr file_segment)
fs::create_directories(subdir); fs::create_directories(subdir);
std::string data(size, '0'); std::string data(size, '0');
file_segment->write(data.data(), size, file_segment->getDownloadOffset()); file_segment->write(data.data(), size, file_segment->getCurrentWriteOffset());
} }
void prepareAndDownload(DB::FileSegmentPtr file_segment) void prepareAndDownload(DB::FileSegmentPtr file_segment)
@ -89,6 +89,7 @@ TEST(FileCache, get)
{ {
if (fs::exists(cache_base_path)) if (fs::exists(cache_base_path))
fs::remove_all(cache_base_path); fs::remove_all(cache_base_path);
fs::create_directories(cache_base_path);
DB::ThreadStatus thread_status; DB::ThreadStatus thread_status;
@ -109,7 +110,7 @@ TEST(FileCache, get)
auto key = cache.hash("key1"); auto key = cache.hash("key1");
{ {
auto holder = cache.getOrSet(key, 0, 10, false); /// Add range [0, 9] auto holder = cache.getOrSet(key, 0, 10, {}); /// Add range [0, 9]
auto segments = fromHolder(holder); auto segments = fromHolder(holder);
/// Range was not present in cache. It should be added in cache as one while file segment. /// Range was not present in cache. It should be added in cache as one while file segment.
ASSERT_EQ(segments.size(), 1); ASSERT_EQ(segments.size(), 1);
@ -138,7 +139,7 @@ TEST(FileCache, get)
{ {
/// Want range [5, 14], but [0, 9] already in cache, so only [10, 14] will be put in cache. /// 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, false); auto holder = cache.getOrSet(key, 5, 10, {});
auto segments = fromHolder(holder); auto segments = fromHolder(holder);
ASSERT_EQ(segments.size(), 2); ASSERT_EQ(segments.size(), 2);
@ -158,14 +159,14 @@ TEST(FileCache, get)
ASSERT_EQ(cache.getUsedCacheSize(), 15); ASSERT_EQ(cache.getUsedCacheSize(), 15);
{ {
auto holder = cache.getOrSet(key, 9, 1, false); /// Get [9, 9] auto holder = cache.getOrSet(key, 9, 1, {}); /// Get [9, 9]
auto segments = fromHolder(holder); auto segments = fromHolder(holder);
ASSERT_EQ(segments.size(), 1); ASSERT_EQ(segments.size(), 1);
assertRange(7, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); assertRange(7, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED);
} }
{ {
auto holder = cache.getOrSet(key, 9, 2, false); /// Get [9, 10] auto holder = cache.getOrSet(key, 9, 2, {}); /// Get [9, 10]
auto segments = fromHolder(holder); auto segments = fromHolder(holder);
ASSERT_EQ(segments.size(), 2); ASSERT_EQ(segments.size(), 2);
assertRange(8, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); assertRange(8, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED);
@ -173,15 +174,15 @@ TEST(FileCache, get)
} }
{ {
auto holder = cache.getOrSet(key, 10, 1, false); /// Get [10, 10] auto holder = cache.getOrSet(key, 10, 1, {}); /// Get [10, 10]
auto segments = fromHolder(holder); auto segments = fromHolder(holder);
ASSERT_EQ(segments.size(), 1); ASSERT_EQ(segments.size(), 1);
assertRange(10, segments[0], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED); assertRange(10, segments[0], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED);
} }
complete(cache.getOrSet(key, 17, 4, false)); /// Get [17, 20] complete(cache.getOrSet(key, 17, 4, {})); /// Get [17, 20]
complete(cache.getOrSet(key, 24, 3, false)); /// Get [24, 26] complete(cache.getOrSet(key, 24, 3, {})); /// Get [24, 26]
/// complete(cache.getOrSet(key, 27, 1, false)); /// Get [27, 27] /// completeWithState(cache.getOrSet(key, 27, 1, false)); /// Get [27, 27]
/// Current cache: [__________][_____] [____] [___][] /// Current cache: [__________][_____] [____] [___][]
/// ^ ^^ ^ ^ ^ ^ ^^^ /// ^ ^^ ^ ^ ^ ^ ^^^
@ -191,7 +192,7 @@ TEST(FileCache, get)
ASSERT_EQ(cache.getUsedCacheSize(), 22); ASSERT_EQ(cache.getUsedCacheSize(), 22);
{ {
auto holder = cache.getOrSet(key, 0, 26, false); /// Get [0, 25] auto holder = cache.getOrSet(key, 0, 26, {}); /// Get [0, 25]
auto segments = fromHolder(holder); auto segments = fromHolder(holder);
ASSERT_EQ(segments.size(), 6); ASSERT_EQ(segments.size(), 6);
@ -225,14 +226,14 @@ TEST(FileCache, get)
/// as max elements size is reached, next attempt to put something in cache should fail. /// 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. /// This will also check that [27, 27] was indeed evicted.
auto holder1 = cache.getOrSet(key, 27, 1, false); auto holder1 = cache.getOrSet(key, 27, 1, {});
auto segments_1 = fromHolder(holder1); /// Get [27, 27] auto segments_1 = fromHolder(holder1); /// Get [27, 27]
ASSERT_EQ(segments_1.size(), 1); ASSERT_EQ(segments_1.size(), 1);
assertRange(17, segments_1[0], DB::FileSegment::Range(27, 27), DB::FileSegment::State::EMPTY); assertRange(17, segments_1[0], DB::FileSegment::Range(27, 27), DB::FileSegment::State::EMPTY);
} }
{ {
auto holder = cache.getOrSet(key, 12, 10, false); /// Get [12, 21] auto holder = cache.getOrSet(key, 12, 10, {}); /// Get [12, 21]
auto segments = fromHolder(holder); auto segments = fromHolder(holder);
ASSERT_EQ(segments.size(), 4); ASSERT_EQ(segments.size(), 4);
@ -256,7 +257,7 @@ TEST(FileCache, get)
ASSERT_EQ(cache.getFileSegmentsNum(), 5); ASSERT_EQ(cache.getFileSegmentsNum(), 5);
{ {
auto holder = cache.getOrSet(key, 23, 5, false); /// Get [23, 28] auto holder = cache.getOrSet(key, 23, 5, {}); /// Get [23, 28]
auto segments = fromHolder(holder); auto segments = fromHolder(holder);
ASSERT_EQ(segments.size(), 3); ASSERT_EQ(segments.size(), 3);
@ -277,12 +278,12 @@ TEST(FileCache, get)
/// 17 21 2324 26 28 /// 17 21 2324 26 28
{ {
auto holder5 = cache.getOrSet(key, 2, 3,false); /// Get [2, 4] auto holder5 = cache.getOrSet(key, 2, 3, {}); /// Get [2, 4]
auto s5 = fromHolder(holder5); auto s5 = fromHolder(holder5);
ASSERT_EQ(s5.size(), 1); ASSERT_EQ(s5.size(), 1);
assertRange(25, s5[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::EMPTY); assertRange(25, s5[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::EMPTY);
auto holder1 = cache.getOrSet(key, 30, 2, false); /// Get [30, 31] auto holder1 = cache.getOrSet(key, 30, 2, {}); /// Get [30, 31]
auto s1 = fromHolder(holder1); auto s1 = fromHolder(holder1);
ASSERT_EQ(s1.size(), 1); ASSERT_EQ(s1.size(), 1);
assertRange(26, s1[0], DB::FileSegment::Range(30, 31), DB::FileSegment::State::EMPTY); assertRange(26, s1[0], DB::FileSegment::Range(30, 31), DB::FileSegment::State::EMPTY);
@ -298,20 +299,20 @@ TEST(FileCache, get)
/// ^ ^ ^ ^ ^ ^ ^ ^ /// ^ ^ ^ ^ ^ ^ ^ ^
/// 2 4 23 24 26 27 30 31 /// 2 4 23 24 26 27 30 31
auto holder2 = cache.getOrSet(key, 23, 1, false); /// Get [23, 23] auto holder2 = cache.getOrSet(key, 23, 1, {}); /// Get [23, 23]
auto s2 = fromHolder(holder2); auto s2 = fromHolder(holder2);
ASSERT_EQ(s2.size(), 1); ASSERT_EQ(s2.size(), 1);
auto holder3 = cache.getOrSet(key, 24, 3, false); /// Get [24, 26] auto holder3 = cache.getOrSet(key, 24, 3, {}); /// Get [24, 26]
auto s3 = fromHolder(holder3); auto s3 = fromHolder(holder3);
ASSERT_EQ(s3.size(), 1); ASSERT_EQ(s3.size(), 1);
auto holder4 = cache.getOrSet(key, 27, 1, false); /// Get [27, 27] auto holder4 = cache.getOrSet(key, 27, 1, {}); /// Get [27, 27]
auto s4 = fromHolder(holder4); auto s4 = fromHolder(holder4);
ASSERT_EQ(s4.size(), 1); ASSERT_EQ(s4.size(), 1);
/// All cache is now unreleasable because pointers are still hold /// All cache is now unreleasable because pointers are still hold
auto holder6 = cache.getOrSet(key, 0, 40, false); auto holder6 = cache.getOrSet(key, 0, 40, {});
auto f = fromHolder(holder6); auto f = fromHolder(holder6);
ASSERT_EQ(f.size(), 9); ASSERT_EQ(f.size(), 9);
@ -332,7 +333,7 @@ TEST(FileCache, get)
} }
{ {
auto holder = cache.getOrSet(key, 2, 3, false); /// Get [2, 4] auto holder = cache.getOrSet(key, 2, 3, {}); /// Get [2, 4]
auto segments = fromHolder(holder); auto segments = fromHolder(holder);
ASSERT_EQ(segments.size(), 1); ASSERT_EQ(segments.size(), 1);
assertRange(31, segments[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::DOWNLOADED); assertRange(31, segments[0], DB::FileSegment::Range(2, 4), DB::FileSegment::State::DOWNLOADED);
@ -343,7 +344,7 @@ TEST(FileCache, get)
/// 2 4 23 24 26 27 30 31 /// 2 4 23 24 26 27 30 31
{ {
auto holder = cache.getOrSet(key, 25, 5, false); /// Get [25, 29] auto holder = cache.getOrSet(key, 25, 5, {}); /// Get [25, 29]
auto segments = fromHolder(holder); auto segments = fromHolder(holder);
ASSERT_EQ(segments.size(), 3); ASSERT_EQ(segments.size(), 3);
@ -367,7 +368,7 @@ TEST(FileCache, get)
DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1); DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1);
thread_status_1.attachQueryContext(query_context_1); thread_status_1.attachQueryContext(query_context_1);
auto holder_2 = cache.getOrSet(key, 25, 5, false); /// Get [25, 29] once again. auto holder_2 = cache.getOrSet(key, 25, 5, {}); /// Get [25, 29] once again.
auto segments_2 = fromHolder(holder_2); auto segments_2 = fromHolder(holder_2);
ASSERT_EQ(segments.size(), 3); ASSERT_EQ(segments.size(), 3);
@ -406,11 +407,11 @@ TEST(FileCache, get)
{ {
/// Now let's check the similar case but getting ERROR state after segment->wait(), when /// Now let's check the similar case but getting ERROR state after segment->wait(), when
/// state is changed not manually via segment->complete(state) but from destructor of holder /// state is changed not manually via segment->completeWithState(state) but from destructor of holder
/// and notify_all() is also called from destructor of holder. /// and notify_all() is also called from destructor of holder.
std::optional<DB::FileSegmentsHolder> holder; std::optional<DB::FileSegmentsHolder> holder;
holder.emplace(cache.getOrSet(key, 3, 23, false)); /// Get [3, 25] holder.emplace(cache.getOrSet(key, 3, 23, {})); /// Get [3, 25]
auto segments = fromHolder(*holder); auto segments = fromHolder(*holder);
ASSERT_EQ(segments.size(), 3); ASSERT_EQ(segments.size(), 3);
@ -436,7 +437,7 @@ TEST(FileCache, get)
DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1); DB::CurrentThread::QueryScope query_scope_holder_1(query_context_1);
thread_status_1.attachQueryContext(query_context_1); thread_status_1.attachQueryContext(query_context_1);
auto holder_2 = cache.getOrSet(key, 3, 23, false); /// Get [3, 25] once again auto holder_2 = cache.getOrSet(key, 3, 23, {}); /// Get [3, 25] once again
auto segments_2 = fromHolder(*holder); auto segments_2 = fromHolder(*holder);
ASSERT_EQ(segments_2.size(), 3); ASSERT_EQ(segments_2.size(), 3);
@ -485,7 +486,7 @@ TEST(FileCache, get)
cache2.initialize(); cache2.initialize();
auto key = cache2.hash("key1"); auto key = cache2.hash("key1");
auto holder1 = cache2.getOrSet(key, 2, 28, false); /// Get [2, 29] auto holder1 = cache2.getOrSet(key, 2, 28, {}); /// Get [2, 29]
auto segments1 = fromHolder(holder1); auto segments1 = fromHolder(holder1);
ASSERT_EQ(segments1.size(), 5); ASSERT_EQ(segments1.size(), 5);
@ -506,7 +507,7 @@ TEST(FileCache, get)
cache2.initialize(); cache2.initialize();
auto key = cache2.hash("key1"); auto key = cache2.hash("key1");
auto holder1 = cache2.getOrSet(key, 0, 25, false); /// Get [0, 24] auto holder1 = cache2.getOrSet(key, 0, 25, {}); /// Get [0, 24]
auto segments1 = fromHolder(holder1); auto segments1 = fromHolder(holder1);
ASSERT_EQ(segments1.size(), 3); ASSERT_EQ(segments1.size(), 3);

View File

@ -2,6 +2,7 @@
#include <DataTypes/NestedUtils.h> #include <DataTypes/NestedUtils.h>
#include <Storages/MergeTree/MergeTreeReaderCompact.h> #include <Storages/MergeTree/MergeTreeReaderCompact.h>
#include <Storages/MergeTree/MergeTreeDataPartWriterCompact.h> #include <Storages/MergeTree/MergeTreeDataPartWriterCompact.h>
#include <Interpreters/Context.h>
#include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h> #include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h>
@ -47,9 +48,11 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader(
const ReadBufferFromFileBase::ProfileCallback & profile_callback) const const ReadBufferFromFileBase::ProfileCallback & profile_callback) const
{ {
auto read_info = std::make_shared<LoadedMergeTreeDataPartInfoForReader>(shared_from_this()); auto read_info = std::make_shared<LoadedMergeTreeDataPartInfoForReader>(shared_from_this());
auto * load_marks_threadpool = reader_settings.read_settings.load_marks_asynchronously ? &read_info->getContext()->getLoadMarksThreadpool() : nullptr;
return std::make_unique<MergeTreeReaderCompact>( return std::make_unique<MergeTreeReaderCompact>(
read_info, columns_to_read, metadata_snapshot, uncompressed_cache, read_info, columns_to_read, metadata_snapshot, uncompressed_cache,
mark_cache, mark_ranges, reader_settings, mark_cache, mark_ranges, reader_settings, load_marks_threadpool,
avg_value_size_hints, profile_callback); avg_value_size_hints, profile_callback);
} }
@ -93,7 +96,7 @@ void MergeTreeDataPartCompact::calculateEachColumnSizes(ColumnSizeByName & /*eac
void MergeTreeDataPartCompact::loadIndexGranularityImpl( void MergeTreeDataPartCompact::loadIndexGranularityImpl(
MergeTreeIndexGranularity & index_granularity_, const MergeTreeIndexGranularityInfo & index_granularity_info_, MergeTreeIndexGranularity & index_granularity_, const MergeTreeIndexGranularityInfo & index_granularity_info_,
const NamesAndTypesList & columns_, const DataPartStoragePtr & data_part_storage_) size_t columns_count, const DataPartStoragePtr & data_part_storage_)
{ {
if (!index_granularity_info_.is_adaptive) if (!index_granularity_info_.is_adaptive)
throw Exception("MergeTreeDataPartCompact cannot be created with non-adaptive granulary.", ErrorCodes::NOT_IMPLEMENTED); throw Exception("MergeTreeDataPartCompact cannot be created with non-adaptive granulary.", ErrorCodes::NOT_IMPLEMENTED);
@ -111,13 +114,13 @@ void MergeTreeDataPartCompact::loadIndexGranularityImpl(
while (!buffer->eof()) while (!buffer->eof())
{ {
/// Skip offsets for columns /// Skip offsets for columns
buffer->seek(columns_.size() * sizeof(MarkInCompressedFile), SEEK_CUR); buffer->seek(columns_count * sizeof(MarkInCompressedFile), SEEK_CUR);
size_t granularity; size_t granularity;
readIntBinary(granularity, *buffer); readIntBinary(granularity, *buffer);
index_granularity_.appendMark(granularity); index_granularity_.appendMark(granularity);
} }
if (index_granularity_.getMarksCount() * index_granularity_info_.getMarkSizeInBytes(columns_.size()) != marks_file_size) if (index_granularity_.getMarksCount() * index_granularity_info_.getMarkSizeInBytes(columns_count) != marks_file_size)
throw Exception("Cannot read all marks from file " + marks_file_path, ErrorCodes::CANNOT_READ_ALL_DATA); throw Exception("Cannot read all marks from file " + marks_file_path, ErrorCodes::CANNOT_READ_ALL_DATA);
index_granularity_.setInitialized(); index_granularity_.setInitialized();
@ -128,7 +131,7 @@ void MergeTreeDataPartCompact::loadIndexGranularity()
if (columns.empty()) if (columns.empty())
throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
loadIndexGranularityImpl(index_granularity, index_granularity_info, columns, data_part_storage); loadIndexGranularityImpl(index_granularity, index_granularity_info, columns.size(), data_part_storage);
} }
bool MergeTreeDataPartCompact::hasColumnFiles(const NameAndTypePair & column) const bool MergeTreeDataPartCompact::hasColumnFiles(const NameAndTypePair & column) const

View File

@ -68,7 +68,7 @@ public:
protected: protected:
static void loadIndexGranularityImpl( static void loadIndexGranularityImpl(
MergeTreeIndexGranularity & index_granularity_, const MergeTreeIndexGranularityInfo & index_granularity_info_, MergeTreeIndexGranularity & index_granularity_, const MergeTreeIndexGranularityInfo & index_granularity_info_,
const NamesAndTypesList & columns_, const DataPartStoragePtr & data_part_storage_); size_t columns_count, const DataPartStoragePtr & data_part_storage_);
private: private:
void checkConsistency(bool require_part_metadata) const override; void checkConsistency(bool require_part_metadata) const override;

View File

@ -1,4 +1,5 @@
#include <Storages/MergeTree/MergeTreeIndexReader.h> #include <Storages/MergeTree/MergeTreeIndexReader.h>
#include <Interpreters/Context.h>
namespace namespace
{ {
@ -15,6 +16,9 @@ std::unique_ptr<MergeTreeReaderStream> makeIndexReader(
UncompressedCache * uncompressed_cache, UncompressedCache * uncompressed_cache,
MergeTreeReaderSettings settings) MergeTreeReaderSettings settings)
{ {
auto context = part->storage.getContext();
auto * load_marks_threadpool = settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr;
return std::make_unique<MergeTreeReaderStream>( return std::make_unique<MergeTreeReaderStream>(
part->data_part_storage, part->data_part_storage,
index->getFileName(), extension, marks_count, index->getFileName(), extension, marks_count,
@ -22,7 +26,7 @@ std::unique_ptr<MergeTreeReaderStream> makeIndexReader(
std::move(settings), mark_cache, uncompressed_cache, std::move(settings), mark_cache, uncompressed_cache,
part->getFileSizeOrZero(index->getFileName() + extension), part->getFileSizeOrZero(index->getFileName() + extension),
&part->index_granularity_info, &part->index_granularity_info,
ReadBufferFromFileBase::ProfileCallback{}, CLOCK_MONOTONIC_COARSE, false); ReadBufferFromFileBase::ProfileCallback{}, CLOCK_MONOTONIC_COARSE, false, load_marks_threadpool);
} }
} }

View File

@ -2,9 +2,19 @@
#include <Storages/MergeTree/MergeTreeData.h> #include <Storages/MergeTree/MergeTreeData.h>
#include <Common/MemoryTrackerBlockerInThread.h> #include <Common/MemoryTrackerBlockerInThread.h>
#include <IO/ReadBufferFromFile.h> #include <IO/ReadBufferFromFile.h>
#include <Common/setThreadName.h>
#include <Common/scope_guard_safe.h>
#include <Common/CurrentMetrics.h>
#include <Common/ThreadPool.h>
#include <utility> #include <utility>
namespace ProfileEvents
{
extern const Event WaitMarksLoadMicroseconds;
extern const Event BackgroundLoadingMarksTasks;
}
namespace DB namespace DB
{ {
@ -23,6 +33,7 @@ MergeTreeMarksLoader::MergeTreeMarksLoader(
const MergeTreeIndexGranularityInfo & index_granularity_info_, const MergeTreeIndexGranularityInfo & index_granularity_info_,
bool save_marks_in_cache_, bool save_marks_in_cache_,
const ReadSettings & read_settings_, const ReadSettings & read_settings_,
ThreadPool * load_marks_threadpool_,
size_t columns_in_mark_) size_t columns_in_mark_)
: data_part_storage(std::move(data_part_storage_)) : data_part_storage(std::move(data_part_storage_))
, mark_cache(mark_cache_) , mark_cache(mark_cache_)
@ -32,13 +43,41 @@ MergeTreeMarksLoader::MergeTreeMarksLoader(
, save_marks_in_cache(save_marks_in_cache_) , save_marks_in_cache(save_marks_in_cache_)
, columns_in_mark(columns_in_mark_) , columns_in_mark(columns_in_mark_)
, read_settings(read_settings_) , read_settings(read_settings_)
, load_marks_threadpool(load_marks_threadpool_)
{ {
if (load_marks_threadpool)
{
future = loadMarksAsync();
}
}
MergeTreeMarksLoader::~MergeTreeMarksLoader()
{
if (future.valid())
{
future.wait();
}
} }
const MarkInCompressedFile & MergeTreeMarksLoader::getMark(size_t row_index, size_t column_index) const MarkInCompressedFile & MergeTreeMarksLoader::getMark(size_t row_index, size_t column_index)
{ {
if (!marks) if (!marks)
loadMarks(); {
Stopwatch watch(CLOCK_MONOTONIC);
if (future.valid())
{
marks = future.get();
future = {};
}
else
{
marks = loadMarks();
}
watch.stop();
ProfileEvents::increment(ProfileEvents::WaitMarksLoadMicroseconds, watch.elapsedMicroseconds());
}
#ifndef NDEBUG #ifndef NDEBUG
if (column_index >= columns_in_mark) if (column_index >= columns_in_mark)
@ -95,28 +134,63 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl()
return res; return res;
} }
void MergeTreeMarksLoader::loadMarks() MarkCache::MappedPtr MergeTreeMarksLoader::loadMarks()
{ {
MarkCache::MappedPtr loaded_marks;
if (mark_cache) if (mark_cache)
{ {
auto key = mark_cache->hash(fs::path(data_part_storage->getFullPath()) / mrk_path); auto key = mark_cache->hash(fs::path(data_part_storage->getFullPath()) / mrk_path);
if (save_marks_in_cache) if (save_marks_in_cache)
{ {
auto callback = [this]{ return loadMarksImpl(); }; auto callback = [this]{ return loadMarksImpl(); };
marks = mark_cache->getOrSet(key, callback); loaded_marks = mark_cache->getOrSet(key, callback);
} }
else else
{ {
marks = mark_cache->get(key); loaded_marks = mark_cache->get(key);
if (!marks) if (!loaded_marks)
marks = loadMarksImpl(); loaded_marks = loadMarksImpl();
} }
} }
else else
marks = loadMarksImpl(); loaded_marks = loadMarksImpl();
if (!marks) if (!loaded_marks)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to load marks: {}", String(fs::path(data_part_storage->getFullPath()) / mrk_path)); {
throw Exception(
ErrorCodes::LOGICAL_ERROR, "Failed to load marks: {}",
(fs::path(data_part_storage->getFullPath()) / mrk_path).string());
}
return loaded_marks;
}
std::future<MarkCache::MappedPtr> MergeTreeMarksLoader::loadMarksAsync()
{
ThreadGroupStatusPtr thread_group;
if (CurrentThread::isInitialized() && CurrentThread::get().getThreadGroup())
thread_group = CurrentThread::get().getThreadGroup();
auto task = std::make_shared<std::packaged_task<MarkCache::MappedPtr()>>([thread_group, this]
{
setThreadName("loadMarksThread");
if (thread_group)
CurrentThread::attachTo(thread_group);
SCOPE_EXIT_SAFE({
if (thread_group)
CurrentThread::detachQuery();
});
ProfileEvents::increment(ProfileEvents::BackgroundLoadingMarksTasks);
return loadMarks();
});
auto task_future = task->get_future();
load_marks_threadpool->scheduleOrThrow([task]{ (*task)(); });
return task_future;
} }
} }

View File

@ -2,11 +2,13 @@
#include <Storages/MergeTree/IDataPartStorage.h> #include <Storages/MergeTree/IDataPartStorage.h>
#include <Storages/MarkCache.h> #include <Storages/MarkCache.h>
#include <IO/ReadSettings.h> #include <IO/ReadSettings.h>
#include <Common/ThreadPool.h>
namespace DB namespace DB
{ {
struct MergeTreeIndexGranularityInfo; struct MergeTreeIndexGranularityInfo;
class Threadpool;
class MergeTreeMarksLoader class MergeTreeMarksLoader
{ {
@ -21,8 +23,11 @@ public:
const MergeTreeIndexGranularityInfo & index_granularity_info_, const MergeTreeIndexGranularityInfo & index_granularity_info_,
bool save_marks_in_cache_, bool save_marks_in_cache_,
const ReadSettings & read_settings_, const ReadSettings & read_settings_,
ThreadPool * load_marks_threadpool_,
size_t columns_in_mark_ = 1); size_t columns_in_mark_ = 1);
~MergeTreeMarksLoader();
const MarkInCompressedFile & getMark(size_t row_index, size_t column_index = 0); const MarkInCompressedFile & getMark(size_t row_index, size_t column_index = 0);
private: private:
@ -36,8 +41,12 @@ private:
MarkCache::MappedPtr marks; MarkCache::MappedPtr marks;
ReadSettings read_settings; ReadSettings read_settings;
void loadMarks(); MarkCache::MappedPtr loadMarks();
std::future<MarkCache::MappedPtr> loadMarksAsync();
MarkCache::MappedPtr loadMarksImpl(); MarkCache::MappedPtr loadMarksImpl();
std::future<MarkCache::MappedPtr> future;
ThreadPool * load_marks_threadpool;
}; };
} }

View File

@ -22,6 +22,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact(
MarkCache * mark_cache_, MarkCache * mark_cache_,
MarkRanges mark_ranges_, MarkRanges mark_ranges_,
MergeTreeReaderSettings settings_, MergeTreeReaderSettings settings_,
ThreadPool * load_marks_threadpool_,
ValueSizeMap avg_value_size_hints_, ValueSizeMap avg_value_size_hints_,
const ReadBufferFromFileBase::ProfileCallback & profile_callback_, const ReadBufferFromFileBase::ProfileCallback & profile_callback_,
clockid_t clock_type_) clockid_t clock_type_)
@ -42,6 +43,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact(
data_part_info_for_read_->getIndexGranularityInfo(), data_part_info_for_read_->getIndexGranularityInfo(),
settings.save_marks_in_cache, settings.save_marks_in_cache,
settings.read_settings, settings.read_settings,
load_marks_threadpool_,
data_part_info_for_read_->getColumns().size()) data_part_info_for_read_->getColumns().size())
{ {
try try

View File

@ -26,6 +26,7 @@ public:
MarkCache * mark_cache_, MarkCache * mark_cache_,
MarkRanges mark_ranges_, MarkRanges mark_ranges_,
MergeTreeReaderSettings settings_, MergeTreeReaderSettings settings_,
ThreadPool * load_marks_threadpool_,
ValueSizeMap avg_value_size_hints_ = {}, ValueSizeMap avg_value_size_hints_ = {},
const ReadBufferFromFileBase::ProfileCallback & profile_callback_ = {}, const ReadBufferFromFileBase::ProfileCallback & profile_callback_ = {},
clockid_t clock_type_ = CLOCK_MONOTONIC_COARSE); clockid_t clock_type_ = CLOCK_MONOTONIC_COARSE);

View File

@ -16,14 +16,19 @@ namespace ErrorCodes
MergeTreeReaderStream::MergeTreeReaderStream( MergeTreeReaderStream::MergeTreeReaderStream(
DataPartStoragePtr data_part_storage_, DataPartStoragePtr data_part_storage_,
const String & path_prefix_, const String & data_file_extension_, size_t marks_count_, const String & path_prefix_,
const String & data_file_extension_,
size_t marks_count_,
const MarkRanges & all_mark_ranges_, const MarkRanges & all_mark_ranges_,
const MergeTreeReaderSettings & settings_, const MergeTreeReaderSettings & settings_,
MarkCache * mark_cache_, MarkCache * mark_cache_,
UncompressedCache * uncompressed_cache_, size_t file_size_, UncompressedCache * uncompressed_cache_,
size_t file_size_,
const MergeTreeIndexGranularityInfo * index_granularity_info_, const MergeTreeIndexGranularityInfo * index_granularity_info_,
const ReadBufferFromFileBase::ProfileCallback & profile_callback_, clockid_t clock_type_, const ReadBufferFromFileBase::ProfileCallback & profile_callback_,
bool is_low_cardinality_dictionary_) clockid_t clock_type_,
bool is_low_cardinality_dictionary_,
ThreadPool * load_marks_cache_threadpool_)
: settings(settings_) : settings(settings_)
, profile_callback(profile_callback_) , profile_callback(profile_callback_)
, clock_type(clock_type_) , clock_type(clock_type_)
@ -45,7 +50,8 @@ MergeTreeReaderStream::MergeTreeReaderStream(
marks_count, marks_count,
*index_granularity_info, *index_granularity_info,
save_marks_in_cache, save_marks_in_cache,
settings.read_settings) settings.read_settings,
load_marks_cache_threadpool_)
{ {
} }

View File

@ -20,13 +20,19 @@ class MergeTreeReaderStream
public: public:
MergeTreeReaderStream( MergeTreeReaderStream(
DataPartStoragePtr data_part_storage_, DataPartStoragePtr data_part_storage_,
const String & path_prefix_, const String & data_file_extension_, size_t marks_count_, const String & path_prefix_,
const String & data_file_extension_,
size_t marks_count_,
const MarkRanges & all_mark_ranges, const MarkRanges & all_mark_ranges,
const MergeTreeReaderSettings & settings_, const MergeTreeReaderSettings & settings_,
MarkCache * mark_cache, UncompressedCache * uncompressed_cache, MarkCache * mark_cache,
size_t file_size_, const MergeTreeIndexGranularityInfo * index_granularity_info_, UncompressedCache * uncompressed_cache,
const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type, size_t file_size_,
bool is_low_cardinality_dictionary_); const MergeTreeIndexGranularityInfo * index_granularity_info_,
const ReadBufferFromFileBase::ProfileCallback & profile_callback,
clockid_t clock_type,
bool is_low_cardinality_dictionary_,
ThreadPool * load_marks_cache_threadpool_);
void seekToMark(size_t index); void seekToMark(size_t index);

View File

@ -6,6 +6,7 @@
#include <DataTypes/NestedUtils.h> #include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeNested.h> #include <DataTypes/DataTypeNested.h>
#include <Interpreters/inplaceBlockConversions.h> #include <Interpreters/inplaceBlockConversions.h>
#include <Interpreters/Context.h>
#include <Storages/MergeTree/IMergeTreeReader.h> #include <Storages/MergeTree/IMergeTreeReader.h>
#include <Storages/MergeTree/MergeTreeDataPartWide.h> #include <Storages/MergeTree/MergeTreeDataPartWide.h>
#include <Common/escapeForFileName.h> #include <Common/escapeForFileName.h>
@ -186,12 +187,15 @@ void MergeTreeReaderWide::addStreams(
has_any_stream = true; has_any_stream = true;
bool is_lc_dict = substream_path.size() > 1 && substream_path[substream_path.size() - 2].type == ISerialization::Substream::Type::DictionaryKeys; bool is_lc_dict = substream_path.size() > 1 && substream_path[substream_path.size() - 2].type == ISerialization::Substream::Type::DictionaryKeys;
auto context = data_part_info_for_read->getContext();
auto * load_marks_threadpool = settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr;
streams.emplace(stream_name, std::make_unique<MergeTreeReaderStream>( streams.emplace(stream_name, std::make_unique<MergeTreeReaderStream>(
data_part_info_for_read->getDataPartStorage(), stream_name, DATA_FILE_EXTENSION, data_part_info_for_read->getDataPartStorage(), stream_name, DATA_FILE_EXTENSION,
data_part_info_for_read->getMarksCount(), all_mark_ranges, settings, mark_cache, data_part_info_for_read->getMarksCount(), all_mark_ranges, settings, mark_cache,
uncompressed_cache, data_part_info_for_read->getFileSizeOrZero(stream_name + DATA_FILE_EXTENSION), uncompressed_cache, data_part_info_for_read->getFileSizeOrZero(stream_name + DATA_FILE_EXTENSION),
&data_part_info_for_read->getIndexGranularityInfo(), &data_part_info_for_read->getIndexGranularityInfo(),
profile_callback, clock_type, is_lc_dict)); profile_callback, clock_type, is_lc_dict, load_marks_threadpool));
}; };
serialization->enumerateStreams(callback); serialization->enumerateStreams(callback);

View File

@ -523,8 +523,7 @@ Chunk StorageEmbeddedRocksDB::getByKeys(
Block StorageEmbeddedRocksDB::getSampleBlock(const Names &) const Block StorageEmbeddedRocksDB::getSampleBlock(const Names &) const
{ {
auto metadata = getInMemoryMetadataPtr(); return getInMemoryMetadataPtr()->getSampleBlock();
return metadata ? metadata->getSampleBlock() : Block();
} }
Chunk StorageEmbeddedRocksDB::getBySerializedKeys( Chunk StorageEmbeddedRocksDB::getBySerializedKeys(

View File

@ -61,6 +61,7 @@ ln -sf $SRC_PATH/users.d/memory_profiler.xml $DEST_SERVER_PATH/users.d/
ln -sf $SRC_PATH/users.d/no_fsync_metadata.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/no_fsync_metadata.xml $DEST_SERVER_PATH/users.d/
ln -sf $SRC_PATH/users.d/filelog.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/filelog.xml $DEST_SERVER_PATH/users.d/
ln -sf $SRC_PATH/users.d/enable_blobs_check.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/enable_blobs_check.xml $DEST_SERVER_PATH/users.d/
ln -sf $SRC_PATH/users.d/marks.xml $DEST_SERVER_PATH/users.d/
# FIXME DataPartsExchange may hang for http_send_timeout seconds # FIXME DataPartsExchange may hang for http_send_timeout seconds
# when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"), # when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"),

View File

@ -0,0 +1,7 @@
<clickhouse>
<profiles>
<default>
<load_marks_asynchronously>1</load_marks_asynchronously>
</default>
</profiles>
</clickhouse>

View File

@ -2,6 +2,7 @@ DROP TABLE IF EXISTS data_01283;
set remote_filesystem_read_method = 'read'; set remote_filesystem_read_method = 'read';
set local_filesystem_read_method = 'pread'; set local_filesystem_read_method = 'pread';
set load_marks_asynchronously = 0;
CREATE TABLE data_01283 engine=MergeTree() CREATE TABLE data_01283 engine=MergeTree()
ORDER BY key ORDER BY key

View File

@ -2,6 +2,7 @@ drop table if exists table_01323_many_parts;
set remote_filesystem_read_method = 'read'; set remote_filesystem_read_method = 'read';
set local_filesystem_read_method = 'pread'; set local_filesystem_read_method = 'pread';
set load_marks_asynchronously = 0;
create table table_01323_many_parts (x UInt64) engine = MergeTree order by x partition by x % 100; create table table_01323_many_parts (x UInt64) engine = MergeTree order by x partition by x % 100;
set max_partitions_per_insert_block = 100; set max_partitions_per_insert_block = 100;

View File

@ -33,6 +33,7 @@ OPTIMIZE TABLE select_final FINAL;
SET remote_filesystem_read_method = 'read'; SET remote_filesystem_read_method = 'read';
SET local_filesystem_read_method = 'pread'; SET local_filesystem_read_method = 'pread';
set load_marks_asynchronously = 0;
SELECT max(x) FROM select_final FINAL; SELECT max(x) FROM select_final FINAL;

View File

@ -30,7 +30,7 @@ EOF
${CLICKHOUSE_CLIENT} -q "SYSTEM STOP MERGES lazy_mark_test" ${CLICKHOUSE_CLIENT} -q "SYSTEM STOP MERGES lazy_mark_test"
${CLICKHOUSE_CLIENT} -q "INSERT INTO lazy_mark_test select number, number % 3, number % 5, number % 10, number % 13, number % 15, number % 17, number % 18, number % 22, number % 25 from numbers(1000000)" ${CLICKHOUSE_CLIENT} -q "INSERT INTO lazy_mark_test select number, number % 3, number % 5, number % 10, number % 13, number % 15, number % 17, number % 18, number % 22, number % 25 from numbers(1000000)"
${CLICKHOUSE_CLIENT} -q "SYSTEM DROP MARK CACHE" ${CLICKHOUSE_CLIENT} -q "SYSTEM DROP MARK CACHE"
${CLICKHOUSE_CLIENT} --log_queries=1 --query_id "${QUERY_ID}" -q "SELECT * FROM lazy_mark_test WHERE n3==11" ${CLICKHOUSE_CLIENT} --log_queries=1 --query_id "${QUERY_ID}" -q "SELECT * FROM lazy_mark_test WHERE n3==11 SETTINGS load_marks_asynchronously=0"
${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS"
${CLICKHOUSE_CLIENT} -q "select ProfileEvents['FileOpen'] from system.query_log where query_id = '${QUERY_ID}' and type = 'QueryFinish' and current_database = currentDatabase()" ${CLICKHOUSE_CLIENT} -q "select ProfileEvents['FileOpen'] from system.query_log where query_id = '${QUERY_ID}' and type = 'QueryFinish' and current_database = currentDatabase()"

View File

@ -0,0 +1,2 @@
Ok
Ok

View File

@ -0,0 +1,51 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS test;"
${CLICKHOUSE_CLIENT} -n -q "
CREATE TABLE test
(
n0 UInt64,
n1 UInt64,
n2 UInt64,
n3 UInt64,
n4 UInt64,
n5 UInt64,
n6 UInt64,
n7 UInt64,
n8 UInt64,
n9 UInt64
)
ENGINE = MergeTree
ORDER BY n0 SETTINGS min_bytes_for_wide_part = 1;"
${CLICKHOUSE_CLIENT} -q "INSERT INTO test select number, number % 3, number % 5, number % 10, number % 13, number % 15, number % 17, number % 18, number % 22, number % 25 from numbers(1000000)"
${CLICKHOUSE_CLIENT} -q "SYSTEM STOP MERGES test"
function test
{
QUERY_ID=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(reverse(reinterpretAsString(generateUUIDv4()))))")
${CLICKHOUSE_CLIENT} -q "SYSTEM DROP MARK CACHE"
${CLICKHOUSE_CLIENT} --query_id "${QUERY_ID}" -q "SELECT * FROM test SETTINGS load_marks_asynchronously=$1 FORMAT Null"
${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS"
result=$(${CLICKHOUSE_CLIENT} -q "SELECT ProfileEvents['WaitMarksLoadMicroseconds'] FROM system.query_log WHERE query_id = '${QUERY_ID}' AND type = 'QueryFinish' AND current_database = currentDatabase()")
if [[ $result -ne 0 ]]; then
echo 'Ok'
else
echo 'F'
fi
result=$(${CLICKHOUSE_CLIENT} -q "SELECT ProfileEvents['BackgroundLoadingMarksTasks'] FROM system.query_log WHERE query_id = '${QUERY_ID}' AND type = 'QueryFinish' AND current_database = currentDatabase()")
if [[ $result -ne 0 ]]; then
echo 'Ok'
else
echo 'F'
fi
}
test 1