ClickHouse/src/IO/RemoteReadBufferCache.cpp

576 lines
20 KiB
C++
Raw Normal View History

2021-11-29 03:30:11 +00:00
#include <fstream>
#include <memory>
#include <unistd.h>
2021-11-05 11:55:30 +00:00
#include <functional>
#include <Core/BackgroundSchedulePool.h>
2021-11-05 11:55:30 +00:00
#include <Poco/Logger.h>
#include <Poco/JSON/JSON.h>
#include <Poco/JSON/Parser.h>
#include <base/logger_useful.h>
2021-11-29 05:01:03 +00:00
#include <base/sleep.h>
2021-11-29 09:01:34 +00:00
#include <base/errnoToString.h>
2021-11-29 03:30:11 +00:00
#include <Common/SipHash.h>
#include <Common/hex.h>
#include <Common/Exception.h>
#include <IO/RemoteReadBufferCache.h>
#include <IO/WriteHelpers.h>
2021-11-05 11:55:30 +00:00
2021-11-29 12:19:36 +00:00
namespace fs = std::filesystem;
2021-11-05 11:55:30 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int BAD_GET;
extern const int LOGICAL_ERROR;
2021-11-29 08:22:43 +00:00
extern const int CANNOT_CREATE_DIRECTORY;
2021-11-05 11:55:30 +00:00
}
2021-11-30 04:05:18 +00:00
bool RemoteFileMetadata::load(const std::filesystem::path & local_path)
2021-11-05 11:55:30 +00:00
{
2021-12-01 06:13:48 +00:00
auto * log = &Poco::Logger::get("RemoteFileMetadata");
2021-11-30 04:05:18 +00:00
if (!std::filesystem::exists(local_path))
2021-11-05 11:55:30 +00:00
{
2021-11-30 04:05:18 +00:00
LOG_ERROR(log, "file path not exists:{}", local_path.string());
return false;
}
std::ifstream meta_fs(local_path.string());
Poco::JSON::Parser meta_data_parser;
auto meta_data_jobj = meta_data_parser.parse(meta_fs).extract<Poco::JSON::Object::Ptr>();
remote_path = meta_data_jobj->get("remote_path").convert<String>();
schema = meta_data_jobj->get("schema").convert<String>();
cluster = meta_data_jobj->get("cluster").convert<String>();
status = static_cast<LocalStatus>(meta_data_jobj->get("status").convert<Int32>());
last_modification_timestamp = meta_data_jobj->get("last_modification_timestamp").convert<UInt64>();
file_size = meta_data_jobj->get("file_size").convert<UInt64>();
meta_fs.close();
return true;
}
2021-12-01 06:13:48 +00:00
void RemoteFileMetadata::save(const std::filesystem::path & local_path) const
2021-11-30 04:05:18 +00:00
{
std::ofstream meta_file(local_path.string(), std::ios::out);
meta_file << toString();
meta_file.close();
}
2021-12-01 06:13:48 +00:00
String RemoteFileMetadata::toString() const
{
2021-11-30 04:05:18 +00:00
Poco::JSON::Object jobj;
jobj.set("schema", schema);
jobj.set("cluster", cluster);
jobj.set("remote_path", remote_path);
jobj.set("status", static_cast<Int32>(status));
jobj.set("last_modification_timestamp", last_modification_timestamp);
jobj.set("file_size", file_size);
std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
jobj.stringify(buf);
return buf.str();
}
std::shared_ptr<RemoteCacheController> RemoteCacheController::recover(const std::filesystem::path & local_path_)
{
auto * log = &Poco::Logger::get("RemoteCacheController");
2021-11-30 04:05:18 +00:00
if (!std::filesystem::exists(local_path_) || !std::filesystem::exists(local_path_ / "data.bin"))
{
LOG_TRACE(log, "Invalid cached directory:{}", local_path_.string());
2021-11-05 11:55:30 +00:00
return nullptr;
}
2021-11-30 04:05:18 +00:00
RemoteFileMetadata remote_file_meta_data;
if (!remote_file_meta_data.load(local_path_ / "meta.txt") || remote_file_meta_data.status != RemoteFileMetadata::DOWNLOADED)
2021-11-05 11:55:30 +00:00
{
2021-12-01 03:32:58 +00:00
LOG_INFO(log, "recover cached file failed. local path:{}, file meta data:{}", local_path_.string(), remote_file_meta_data.toString());
2021-11-05 11:55:30 +00:00
return nullptr;
}
auto cache_controller = std::make_shared<RemoteCacheController>(nullptr, remote_file_meta_data, local_path_, 0, nullptr);
2021-11-30 04:05:18 +00:00
cache_controller->current_offset = remote_file_meta_data.file_size;
2021-11-05 11:55:30 +00:00
2021-11-30 04:05:18 +00:00
RemoteReadBufferCache::instance().updateTotalSize(cache_controller->file_meta_data.file_size);
2021-11-29 05:01:03 +00:00
return cache_controller;
2021-11-05 11:55:30 +00:00
}
RemoteCacheController::RemoteCacheController(
ContextPtr context,
2021-11-30 04:05:18 +00:00
const RemoteFileMetadata & file_meta_data_,
const std::filesystem::path & local_path_,
2021-11-11 11:50:08 +00:00
size_t cache_bytes_before_flush_,
2021-11-30 04:05:18 +00:00
std::shared_ptr<ReadBuffer> read_buffer_)
: file_meta_data(file_meta_data_)
2021-11-29 05:01:03 +00:00
, local_path(local_path_)
, valid(true)
, local_cache_bytes_read_before_flush(cache_bytes_before_flush_)
, current_offset(0)
2021-11-29 12:19:36 +00:00
, remote_read_buffer(read_buffer_)
2021-11-05 11:55:30 +00:00
{
2021-11-29 12:19:36 +00:00
/// readbuffer == nullptr if `RemoteCacheController` is created in `initOnce`, when metadata and local cache already exist.
if (remote_read_buffer)
2021-11-05 11:55:30 +00:00
{
// setup local files
2021-11-29 08:22:43 +00:00
out_file = std::make_unique<std::ofstream>(fs::path(local_path_) / "data.bin", std::ios::out | std::ios::binary);
2021-11-05 11:55:30 +00:00
out_file->flush();
2021-11-30 04:05:18 +00:00
file_meta_data.save(local_path_ / "meta.txt");
download_task_holder = context->getSchedulePool().createTask("download remote file", [this]{ this->backgroundDownload(); });
download_task_holder->activateAndSchedule();
2021-11-05 11:55:30 +00:00
}
}
RemoteReadBufferCacheError RemoteCacheController::waitMoreData(size_t start_offset_, size_t end_offset_)
2021-11-05 11:55:30 +00:00
{
std::unique_lock lock{mutex};
2021-11-30 04:05:18 +00:00
if (file_meta_data.status == RemoteFileMetadata::DOWNLOADED)
2021-11-05 11:55:30 +00:00
{
// finish reading
if (start_offset_ >= current_offset)
{
lock.unlock();
return RemoteReadBufferCacheError::END_OF_FILE;
2021-11-05 11:55:30 +00:00
}
}
else // block until more data is ready
{
if (current_offset >= end_offset_)
{
lock.unlock();
return RemoteReadBufferCacheError::OK;
2021-11-05 11:55:30 +00:00
}
else
2021-11-30 04:05:18 +00:00
more_data_signal.wait(lock, [this, end_offset_] { return this->file_meta_data.status == RemoteFileMetadata::DOWNLOADED || current_offset >= end_offset_; });
2021-11-05 11:55:30 +00:00
}
lock.unlock();
return RemoteReadBufferCacheError::OK;
2021-11-05 11:55:30 +00:00
}
2021-11-30 04:05:18 +00:00
void RemoteCacheController::backgroundDownload()
2021-11-05 11:55:30 +00:00
{
file_meta_data.status = RemoteFileMetadata::DOWNLOADING;
size_t before_unflush_bytes = 0;
size_t total_bytes = 0;
while (!remote_read_buffer->eof())
2021-11-29 05:01:03 +00:00
{
size_t bytes = remote_read_buffer->available();
out_file->write(remote_read_buffer->position(), bytes);
remote_read_buffer->position() += bytes;
total_bytes += bytes;
before_unflush_bytes += bytes;
if (before_unflush_bytes >= local_cache_bytes_read_before_flush)
2021-11-05 11:55:30 +00:00
{
std::unique_lock lock(mutex);
current_offset += total_bytes;
total_bytes = 0;
flush();
lock.unlock();
more_data_signal.notify_all();
before_unflush_bytes = 0;
2021-11-05 11:55:30 +00:00
}
}
std::unique_lock lock(mutex);
current_offset += total_bytes;
file_meta_data.status = RemoteFileMetadata::DOWNLOADED;
flush(true);
out_file->close();
out_file.reset();
remote_read_buffer.reset();
lock.unlock();
more_data_signal.notify_all();
RemoteReadBufferCache::instance().updateTotalSize(file_meta_data.file_size);
LOG_TRACE(log, "Finish download into local path: {}, file meta data:{} ", local_path.string(), file_meta_data.toString());
2021-11-05 11:55:30 +00:00
}
2021-11-30 04:05:18 +00:00
void RemoteCacheController::flush(bool need_flush_meta_data_)
2021-11-05 11:55:30 +00:00
{
if (out_file)
2021-11-05 11:55:30 +00:00
{
out_file->flush();
}
2021-11-11 11:50:08 +00:00
2021-11-30 04:05:18 +00:00
if (!need_flush_meta_data_)
2021-11-11 11:50:08 +00:00
return;
2021-11-05 11:55:30 +00:00
2021-11-30 04:05:18 +00:00
file_meta_data.save(local_path / "meta.txt");
2021-11-05 11:55:30 +00:00
}
RemoteCacheController::~RemoteCacheController() = default;
2021-11-29 05:01:03 +00:00
2021-11-05 11:55:30 +00:00
void RemoteCacheController::close()
{
2021-11-30 04:05:18 +00:00
// delete directory
LOG_TRACE(log, "Removing all local cache. local path: {}, file meta data:{}", local_path.string(), file_meta_data.toString());
std::filesystem::remove_all(local_path);
2021-11-05 11:55:30 +00:00
}
std::pair<FILE *, std::filesystem::path> RemoteCacheController::allocFile()
2021-11-05 11:55:30 +00:00
{
std::filesystem::path result_local_path = local_path / "data.bin";
2021-11-29 05:01:03 +00:00
FILE * fs = fopen(result_local_path.string().c_str(), "r");
2021-11-29 09:01:34 +00:00
if (!fs)
2021-11-30 04:05:18 +00:00
throw Exception(ErrorCodes::BAD_GET, "Alloc file failed, error code: {} local path: {}", errno, local_path.string());
2021-11-29 05:01:03 +00:00
2021-11-05 11:55:30 +00:00
std::lock_guard lock{mutex};
opened_file_streams.insert(fs);
return {fs, result_local_path};
2021-11-05 11:55:30 +00:00
}
2021-11-29 12:19:36 +00:00
void RemoteCacheController::deallocFile(FILE * file_stream)
2021-11-05 11:55:30 +00:00
{
{
std::lock_guard lock{mutex};
2021-11-29 12:19:36 +00:00
auto it = opened_file_streams.find(file_stream);
2021-11-05 11:55:30 +00:00
if (it == opened_file_streams.end())
2021-11-29 12:19:36 +00:00
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Try to deallocate file with invalid handler remote path: {}, local path: {}",
2021-11-30 04:05:18 +00:00
file_meta_data.remote_path,
local_path.string());
2021-11-05 11:55:30 +00:00
opened_file_streams.erase(it);
}
2021-11-29 12:19:36 +00:00
fclose(file_stream);
2021-11-05 11:55:30 +00:00
}
LocalCachedFileReader::LocalCachedFileReader(RemoteCacheController * cache_controller_)
2021-11-29 09:01:34 +00:00
: cache_controller(cache_controller_)
, file_size(cache_controller_->getFileMetaData().file_size)
2021-11-29 09:01:34 +00:00
, offset(0)
2021-11-05 11:55:30 +00:00
{
2021-11-29 09:01:34 +00:00
std::tie(file_stream, local_path) = cache_controller->allocFile();
2021-11-05 11:55:30 +00:00
}
2021-11-29 09:01:34 +00:00
2021-11-05 11:55:30 +00:00
LocalCachedFileReader::~LocalCachedFileReader()
{
2021-11-29 09:01:34 +00:00
cache_controller->deallocFile(file_stream);
2021-11-05 11:55:30 +00:00
}
size_t LocalCachedFileReader::read(char * buf, size_t size)
{
2021-11-29 08:22:43 +00:00
auto wret = cache_controller->waitMoreData(offset, offset + size);
if (wret != RemoteReadBufferCacheError::OK)
2021-11-05 11:55:30 +00:00
return 0;
std::lock_guard lock(mutex);
2021-11-29 09:01:34 +00:00
auto ret_size = fread(buf, 1, size, file_stream);
2021-11-05 11:55:30 +00:00
offset += ret_size;
return ret_size;
}
off_t LocalCachedFileReader::seek(off_t off)
{
2021-11-30 11:05:22 +00:00
cache_controller->waitMoreData(off, 0);
2021-11-05 11:55:30 +00:00
std::lock_guard lock(mutex);
2021-11-29 09:01:34 +00:00
auto ret = fseek(file_stream, off, SEEK_SET);
if (ret < 0)
throw Exception(
ErrorCodes::BAD_ARGUMENTS, "Seek file {} with size {} to offset {} failed: {}", getPath().string(), getSize(), off, errnoToString(errno));
2021-11-29 09:01:34 +00:00
2021-11-05 11:55:30 +00:00
offset = off;
return off;
}
2021-11-29 12:19:36 +00:00
2021-11-29 09:01:34 +00:00
size_t LocalCachedFileReader::getSize()
2021-11-05 11:55:30 +00:00
{
2021-11-29 09:01:34 +00:00
return file_size;
2021-11-05 11:55:30 +00:00
}
// the size need be equal to the original buffer
2021-11-29 12:19:36 +00:00
RemoteReadBuffer::RemoteReadBuffer(size_t buff_size) : BufferWithOwnMemory<SeekableReadBufferWithSize>(buff_size)
2021-11-05 11:55:30 +00:00
{
}
RemoteReadBuffer::~RemoteReadBuffer() = default;
std::unique_ptr<RemoteReadBuffer> RemoteReadBuffer::create(ContextPtr context, const RemoteFileMetadata & remote_file_meta, std::unique_ptr<ReadBuffer> read_buffer)
2021-11-05 11:55:30 +00:00
{
2021-11-29 04:17:22 +00:00
auto * log = &Poco::Logger::get("RemoteReadBuffer");
2021-11-05 11:55:30 +00:00
size_t buff_size = DBMS_DEFAULT_BUFFER_SIZE;
2021-11-29 05:01:03 +00:00
if (read_buffer)
buff_size = read_buffer->internalBuffer().size();
2021-11-05 11:55:30 +00:00
/*
* in the new implement of ReadBufferFromHDFS, buffer size is 0.
*
* in the common case, we don't read bytes from readbuffer directly, so set buff_size = DBMS_DEFAULT_BUFFER_SIZE
* is OK.
*
* we need be careful with the case without local file reader.
*/
if (buff_size == 0)
buff_size = DBMS_DEFAULT_BUFFER_SIZE;
2021-11-30 04:05:18 +00:00
const auto & remote_path = remote_file_meta.remote_path;
2021-11-29 05:01:03 +00:00
auto remote_read_buffer = std::make_unique<RemoteReadBuffer>(buff_size);
auto * raw_rbp = read_buffer.release();
2021-11-05 11:55:30 +00:00
std::shared_ptr<ReadBuffer> srb(raw_rbp);
RemoteReadBufferCacheError error;
2021-11-05 11:55:30 +00:00
std::tie(remote_read_buffer->file_reader, error) = RemoteReadBufferCache::instance().createReader(context, remote_file_meta, srb);
2021-11-29 05:01:03 +00:00
if (remote_read_buffer->file_reader == nullptr)
2021-11-05 11:55:30 +00:00
{
2021-11-29 09:01:34 +00:00
LOG_ERROR(log, "Failed to allocate local file for remote path: {}, reason: {}", remote_path, error);
2021-11-29 12:19:36 +00:00
remote_read_buffer->original_read_buffer = srb;
2021-11-05 11:55:30 +00:00
}
2021-11-29 05:01:03 +00:00
return remote_read_buffer;
2021-11-05 11:55:30 +00:00
}
bool RemoteReadBuffer::nextImpl()
{
2021-11-29 05:01:03 +00:00
if (file_reader)
2021-11-05 11:55:30 +00:00
{
int bytes_read = file_reader->read(internal_buffer.begin(), internal_buffer.size());
if (bytes_read)
working_buffer.resize(bytes_read);
else
return false;
}
else
2021-11-05 11:55:30 +00:00
{
2021-11-29 12:19:36 +00:00
// In the case we cannot use local cache, read from the original readbuffer directly
if (!original_read_buffer)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Original read buffer is not initialized. It's a bug");
2021-11-29 09:01:34 +00:00
2021-11-29 12:19:36 +00:00
auto status = original_read_buffer->next();
// We don't need to worry about the memory buffer allocated in RemoteReadBuffer, since it is owned by
2021-11-05 11:55:30 +00:00
// BufferWithOwnMemory, BufferWithOwnMemory would release it.
if (status)
2021-11-29 12:19:36 +00:00
BufferBase::set(original_read_buffer->buffer().begin(), original_read_buffer->buffer().size(), original_read_buffer->offset());
2021-11-05 11:55:30 +00:00
return status;
}
return true;
}
off_t RemoteReadBuffer::seek(off_t offset, int whence)
{
off_t pos_in_file = file_reader->getOffset();
off_t new_pos;
if (whence == SEEK_SET)
new_pos = offset;
else if (whence == SEEK_CUR)
2021-11-29 06:50:33 +00:00
new_pos = pos_in_file - available() + offset;
2021-11-05 11:55:30 +00:00
else
2021-11-29 09:01:34 +00:00
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expects SEEK_SET or SEEK_CUR as whence but given {}", whence);
2021-11-05 11:55:30 +00:00
/// Position is unchanged.
2021-11-29 06:50:33 +00:00
if (off_t(new_pos + available()) == pos_in_file)
2021-11-05 11:55:30 +00:00
return new_pos;
if (new_pos <= pos_in_file && new_pos >= pos_in_file - static_cast<off_t>(working_buffer.size()))
{
/// Position is still inside buffer.
pos = working_buffer.begin() + (new_pos - (pos_in_file - working_buffer.size()));
return new_pos;
}
pos = working_buffer.end();
auto ret_off = file_reader->seek(new_pos);
return ret_off;
}
off_t RemoteReadBuffer::getPosition()
{
return file_reader->getOffset() - available();
2021-11-05 11:55:30 +00:00
}
RemoteReadBufferCache::RemoteReadBufferCache() = default;
RemoteReadBufferCache::~RemoteReadBufferCache() = default;
2021-11-05 11:55:30 +00:00
RemoteReadBufferCache & RemoteReadBufferCache::instance()
{
static RemoteReadBufferCache instance;
return instance;
}
2021-11-30 04:05:18 +00:00
void RemoteReadBufferCache::recoverCachedFilesMetaData(
2021-11-29 08:22:43 +00:00
const fs::path & current_path,
2021-11-29 03:30:11 +00:00
size_t current_depth,
2021-11-30 04:05:18 +00:00
size_t max_depth)
{
if (current_depth >= max_depth)
{
2021-11-29 08:22:43 +00:00
for (auto const & dir : fs::directory_iterator{current_path})
{
2021-11-30 04:05:18 +00:00
String path = dir.path();
auto cache_controller = RemoteCacheController::recover(path);
if (!cache_controller)
continue;
2021-11-29 03:30:11 +00:00
auto & cell = caches[path];
cell.cache_controller = cache_controller;
cell.key_iterator = keys.insert(keys.end(), path);
}
return;
}
2021-11-29 08:22:43 +00:00
for (auto const & dir : fs::directory_iterator{current_path})
{
2021-11-30 04:05:18 +00:00
recoverCachedFilesMetaData(dir.path(), current_depth + 1, max_depth);
}
}
void RemoteReadBufferCache::recoverTask()
{
2021-11-30 04:05:18 +00:00
std::lock_guard lock(mutex);
recoverCachedFilesMetaData(root_dir, 1, 2);
initialized = true;
LOG_TRACE(log, "Recovered from directory:{}", root_dir);
}
2021-11-29 03:30:11 +00:00
void RemoteReadBufferCache::initOnce(
ContextPtr context,
const String & root_dir_, size_t limit_size_, size_t bytes_read_before_flush_)
2021-11-05 11:55:30 +00:00
{
2021-11-29 04:01:52 +00:00
LOG_INFO(
2021-11-29 08:22:43 +00:00
log, "Initializing local cache for remote data sources. Local cache root path: {}, cache size limit: {}", root_dir_, limit_size_);
root_dir = root_dir_;
2021-11-05 11:55:30 +00:00
limit_size = limit_size_;
2021-11-11 11:50:08 +00:00
local_cache_bytes_read_before_flush = bytes_read_before_flush_;
2021-11-05 11:55:30 +00:00
2021-11-29 08:22:43 +00:00
/// create if root_dir not exists
if (!fs::exists(fs::path(root_dir) / ""))
{
2021-11-29 08:22:43 +00:00
std::error_code ec;
bool success = fs::create_directories(fs::path(root_dir) / "", ec);
if (!success)
throw Exception(
ErrorCodes::CANNOT_CREATE_DIRECTORY, "Failed to create directories, error code:{} reason:{}", ec.value(), ec.message());
}
2021-11-29 08:22:43 +00:00
recover_task_holder = context->getSchedulePool().createTask("recover local cache meta data for remote files", [this]{ recoverTask(); });
recover_task_holder->activateAndSchedule();
2021-11-05 11:55:30 +00:00
}
2021-11-29 08:22:43 +00:00
String RemoteReadBufferCache::calculateLocalPath(const RemoteFileMetadata & meta) const
2021-11-05 11:55:30 +00:00
{
2021-11-30 04:05:18 +00:00
String full_path = meta.schema + ":" + meta.cluster + ":" + meta.remote_path;
2021-11-15 02:09:21 +00:00
UInt128 hashcode = sipHash128(full_path.c_str(), full_path.size());
2021-11-29 08:22:43 +00:00
String hashcode_str = getHexUIntLowercase(hashcode);
return fs::path(root_dir) / hashcode_str.substr(0, 3) / hashcode_str;
2021-11-05 11:55:30 +00:00
}
2021-11-29 05:01:03 +00:00
std::pair<std::shared_ptr<LocalCachedFileReader>, RemoteReadBufferCacheError>
RemoteReadBufferCache::createReader(ContextPtr context, const RemoteFileMetadata & remote_file_meta, std::shared_ptr<ReadBuffer> & read_buffer)
2021-11-05 11:55:30 +00:00
{
2021-11-30 04:05:18 +00:00
LOG_TRACE(log, "createReader. {} {} {}", remote_file_meta.remote_path, remote_file_meta.last_modification_timestamp, remote_file_meta.file_size);
2021-11-15 11:08:58 +00:00
// If something is wrong on startup, rollback to read from the original ReadBuffer
2021-11-29 05:01:03 +00:00
if (!isInitialized())
2021-11-05 11:55:30 +00:00
{
2021-11-30 02:49:57 +00:00
LOG_ERROR(log, "RemoteReadBufferCache has not been initialized");
return {nullptr, RemoteReadBufferCacheError::NOT_INIT};
2021-11-05 11:55:30 +00:00
}
2021-11-30 04:05:18 +00:00
auto remote_path = remote_file_meta.remote_path;
const auto & last_modification_timestamp = remote_file_meta.last_modification_timestamp;
auto local_path = calculateLocalPath(remote_file_meta);
2021-11-05 11:55:30 +00:00
std::lock_guard lock(mutex);
auto cache_iter = caches.find(local_path);
if (cache_iter != caches.end())
2021-11-05 11:55:30 +00:00
{
// if the file has been update on remote side, we need to redownload it
if (cache_iter->second.cache_controller->getLastModificationTimestamp() != last_modification_timestamp)
2021-11-05 11:55:30 +00:00
{
2021-11-29 03:30:11 +00:00
LOG_TRACE(
log,
2021-11-29 06:50:33 +00:00
"Remote file ({}) has been updated. Last saved modification time: {}, actual last modification time: {}",
remote_path,
std::to_string(cache_iter->second.cache_controller->getLastModificationTimestamp()),
std::to_string(last_modification_timestamp));
cache_iter->second.cache_controller->markInvalid();
2021-11-05 11:55:30 +00:00
}
else
{
// move the key to the list end
keys.splice(keys.end(), keys, cache_iter->second.key_iterator);
2021-11-29 03:30:11 +00:00
return {
std::make_shared<LocalCachedFileReader>(cache_iter->second.cache_controller.get()),
2021-11-29 03:30:11 +00:00
RemoteReadBufferCacheError::OK};
2021-11-05 11:55:30 +00:00
}
}
auto clear_ret = clearLocalCache();
cache_iter = caches.find(local_path);
if (cache_iter != caches.end())
2021-11-05 11:55:30 +00:00
{
if (cache_iter->second.cache_controller->isValid())
2021-11-05 11:55:30 +00:00
{
keys.splice(keys.end(), keys, cache_iter->second.key_iterator);
2021-11-29 03:30:11 +00:00
return {
std::make_shared<LocalCachedFileReader>(cache_iter->second.cache_controller.get()),
2021-11-29 03:30:11 +00:00
RemoteReadBufferCacheError::OK};
2021-11-05 11:55:30 +00:00
}
else
{
// maybe someone is holding this file
2021-12-01 03:03:33 +00:00
LOG_INFO(log, "The remote file {} has been updated, but the previous readers do not finish reading.",
remote_path);
return {nullptr, RemoteReadBufferCacheError::FILE_INVALID};
2021-11-05 11:55:30 +00:00
}
}
// reach the disk capacity limit
if (!clear_ret)
{
2021-11-29 06:50:33 +00:00
LOG_INFO(log, "Reached local cache capacity limit size ({})", limit_size);
return {nullptr, RemoteReadBufferCacheError::DISK_FULL};
2021-11-05 11:55:30 +00:00
}
2021-11-29 08:22:43 +00:00
fs::create_directories(local_path);
2021-11-05 11:55:30 +00:00
// pass a session context into RemoteCacheController is not a good idea
2021-11-29 06:50:33 +00:00
auto cache_controller
= std::make_shared<RemoteCacheController>(context->getGlobalContext(), remote_file_meta, local_path, local_cache_bytes_read_before_flush, read_buffer);
2021-11-29 06:50:33 +00:00
CacheCell cache_cell;
cache_cell.cache_controller = cache_controller;
cache_cell.key_iterator = keys.insert(keys.end(), local_path);
caches[local_path] = cache_cell;
2021-11-05 11:55:30 +00:00
return {std::make_shared<LocalCachedFileReader>(cache_controller.get()), RemoteReadBufferCacheError::OK};
2021-11-05 11:55:30 +00:00
}
bool RemoteReadBufferCache::clearLocalCache()
{
// clear closable cache from the list head
2021-11-05 11:55:30 +00:00
for (auto it = keys.begin(); it != keys.end();)
{
auto cache_it = caches.find(*it);
if (cache_it == caches.end())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Found no entry in local cache with key: {}", *it);
2021-11-30 10:23:24 +00:00
2021-11-29 06:50:33 +00:00
auto cache_controller = cache_it->second.cache_controller;
if (!cache_controller->isValid() && cache_controller->closable())
2021-11-05 11:55:30 +00:00
{
2021-11-29 09:01:34 +00:00
LOG_TRACE(log, "Clear invalid cache entry with key {} from local cache", *it);
2021-11-29 03:30:11 +00:00
total_size
= total_size > cache_it->second.cache_controller->size() ? total_size - cache_it->second.cache_controller->size() : 0;
2021-11-29 06:50:33 +00:00
cache_controller->close();
2021-11-05 11:55:30 +00:00
it = keys.erase(it);
caches.erase(cache_it);
continue;
2021-11-05 11:55:30 +00:00
}
2021-11-29 09:01:34 +00:00
// if enough disk space is release, just to iterate the remained caches and clear the invalid ones.
if (total_size > limit_size && cache_controller->closable())
2021-11-05 11:55:30 +00:00
{
total_size = total_size > cache_controller->size() ? total_size - cache_controller->size() : 0;
cache_controller->close();
caches.erase(cache_it);
2021-11-05 11:55:30 +00:00
it = keys.erase(it);
2021-11-29 03:30:11 +00:00
LOG_TRACE(
log,
"clear local file {} for {}. key size:{}. next{}",
cache_controller->getLocalPath().string(),
cache_controller->getRemotePath(),
2021-11-29 03:30:11 +00:00
keys.size(),
*it);
2021-11-05 11:55:30 +00:00
}
else
it++;
2021-11-05 11:55:30 +00:00
}
2021-11-29 09:01:34 +00:00
LOG_TRACE(log, "After clear local cache, keys size:{}, total_size:{}, limit size:{}", keys.size(), total_size, limit_size);
return total_size < limit_size;
2021-11-05 11:55:30 +00:00
}
}