ClickHouse/src/IO/RemoteReadBufferCache.cpp

472 lines
17 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 LOGICAL_ERROR;
2021-11-29 08:22:43 +00:00
extern const int CANNOT_CREATE_DIRECTORY;
2021-11-05 11:55:30 +00:00
}
bool RemoteCacheController::loadInnerInformation(const fs::path & file_path)
2021-11-05 11:55:30 +00:00
{
if (!fs::exists(file_path))
2021-11-30 04:05:18 +00:00
return false;
std::ifstream info_file(file_path);
Poco::JSON::Parser info_parser;
2021-12-06 10:19:36 +00:00
auto info_json = info_parser.parse(info_file).extract<Poco::JSON::Object::Ptr>();
file_status = static_cast<LocalFileStatus>(info_json->get("file_status").convert<Int32>());
metadata_class = info_json->get("metadata_class").convert<String>();
info_file.close();
2021-11-30 04:05:18 +00:00
return true;
}
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_ / "data.bin"))
2021-11-30 04:05:18 +00:00
{
LOG_TRACE(log, "Invalid cached directory:{}", local_path_.string());
2021-11-05 11:55:30 +00:00
return nullptr;
}
auto cache_controller = std::make_shared<RemoteCacheController>(nullptr, local_path_, 0);
if (!cache_controller->loadInnerInformation(local_path_ / "info.txt")
|| cache_controller->file_status != DOWNLOADED)
{
LOG_INFO(log, "Recover cached file failed. local path:{}", local_path_.string());
return nullptr;
}
2021-12-06 11:59:15 +00:00
cache_controller->file_metadata_ptr = RemoteFileMetadataFactory::instance().get(cache_controller->metadata_class);
2021-12-06 10:19:36 +00:00
if (!cache_controller->file_metadata_ptr)
{
// do not load this invalid cached file and clear it. the clear action is in
2021-12-06 10:19:36 +00:00
// RemoteReadBufferCache::recoverCachedFilesMetadata(), because deleting directories during iteration will
// cause unexpected behaviors
LOG_ERROR(log, "Cannot create the meta data class : {}. The cached file is invalid and will be remove. path:{}",
2021-12-06 10:19:36 +00:00
cache_controller->metadata_class,
local_path_.string());
return nullptr;
}
2021-12-06 10:19:36 +00:00
std::ifstream metadata_file(local_path_ / "metadata.txt");
if (!cache_controller->file_metadata_ptr->fromString(std::string((std::istreambuf_iterator<char>(metadata_file)),
std::istreambuf_iterator<char>())))
2021-11-05 11:55:30 +00:00
{
LOG_ERROR(log, "Cannot load the meta data. The cached file is invalid and will be remove. path:{}",
local_path_.string());
2021-11-05 11:55:30 +00:00
return nullptr;
}
cache_controller->current_offset = fs::file_size(local_path_ / "data.bin");
2021-11-05 11:55:30 +00:00
2021-12-06 10:19:36 +00:00
RemoteReadBufferCache::instance().updateTotalSize(cache_controller->file_metadata_ptr->getFileSize());
2021-11-29 05:01:03 +00:00
return cache_controller;
2021-11-05 11:55:30 +00:00
}
RemoteCacheController::RemoteCacheController(
2021-12-06 10:19:36 +00:00
IRemoteFileMetadataPtr file_metadata_,
2021-11-30 04:05:18 +00:00
const std::filesystem::path & local_path_,
size_t cache_bytes_before_flush_)
2021-12-06 10:19:36 +00:00
: file_metadata_ptr(file_metadata_)
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-05 11:55:30 +00:00
{
2021-12-06 10:19:36 +00:00
// on recover, file_metadata_ptr is null, but it will be allocated after loading from metadata.txt
// when we allocate a whole new file cache file_metadata_ptr must not be null.
if (file_metadata_ptr)
2021-11-05 11:55:30 +00:00
{
2021-12-06 10:19:36 +00:00
auto metadata_file_writer = std::make_unique<WriteBufferFromFile>((local_path_ / "metadata.txt").string());
auto str_buf = file_metadata_ptr->toString();
metadata_file_writer->write(str_buf.c_str(), str_buf.size());
metadata_file_writer->close();
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};
if (file_status == 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
more_data_signal.wait(lock, [this, end_offset_] { return file_status == 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-12-06 10:19:36 +00:00
bool RemoteCacheController::checkFileChanged(IRemoteFileMetadataPtr file_metadata_)
{
2021-12-06 10:19:36 +00:00
return !(file_metadata_ptr->getVersion() == file_metadata_->getVersion());
}
2021-12-06 10:19:36 +00:00
void RemoteCacheController::startBackgroundDownload(std::unique_ptr<ReadBuffer> in_readbuffer_, BackgroundSchedulePool & thread_pool)
{
data_file_writer = std::make_unique<WriteBufferFromFile>((fs::path(local_path) / "data.bin").string());
flush(true);
2021-12-06 10:19:36 +00:00
ReadBufferPtr in_readbuffer(in_readbuffer_.release());
download_task_holder = thread_pool.createTask("download remote file",
2021-12-06 10:19:36 +00:00
[this, in_readbuffer]{ backgroundDownload(in_readbuffer); });
download_task_holder->activateAndSchedule();
}
2021-12-06 10:19:36 +00:00
void RemoteCacheController::backgroundDownload(ReadBufferPtr remote_read_buffer)
2021-11-05 11:55:30 +00:00
{
file_status = 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();
data_file_writer->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_status = DOWNLOADED;
flush(true);
data_file_writer.reset();
lock.unlock();
more_data_signal.notify_all();
2021-12-06 10:19:36 +00:00
RemoteReadBufferCache::instance().updateTotalSize(file_metadata_ptr->getFileSize());
LOG_TRACE(log, "Finish download into local path: {}, file meta data:{} ", local_path.string(), file_metadata_ptr->toString());
2021-11-05 11:55:30 +00:00
}
void RemoteCacheController::flush(bool need_flush_status)
2021-11-05 11:55:30 +00:00
{
if (data_file_writer)
2021-11-05 11:55:30 +00:00
{
data_file_writer->sync();
2021-11-05 11:55:30 +00:00
}
if (need_flush_status)
{
2021-12-06 10:19:36 +00:00
auto file_writer = std::make_unique<WriteBufferFromFile>(local_path / "info.txt");
Poco::JSON::Object jobj;
jobj.set("file_status", static_cast<Int32>(file_status));
2021-12-06 10:19:36 +00:00
jobj.set("metadata_class", metadata_class);
std::stringstream buf; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
jobj.stringify(buf);
2021-12-06 10:19:36 +00:00
file_writer->write(buf.str().c_str(), buf.str().size());
file_writer->close();
}
2021-11-05 11:55:30 +00:00
}
RemoteCacheController::~RemoteCacheController()
{
if (download_task_holder)
download_task_holder->deactivate();
}
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 the local cache. local path: {}", local_path.string());
2021-11-30 04:05:18 +00:00
std::filesystem::remove_all(local_path);
2021-11-05 11:55:30 +00:00
}
std::unique_ptr<ReadBufferFromFileBase> RemoteCacheController::allocFile()
2021-11-05 11:55:30 +00:00
{
ReadSettings settings;
settings.local_fs_prefetch = false;
settings.local_fs_method = LocalFSReadMethod::read;
auto file_buffer = createReadBufferFromFileBase((local_path / "data.bin").string(), settings);
2021-11-29 05:01:03 +00:00
if (file_buffer)
{
std::lock_guard lock{mutex};
opened_file_buffer_refs.insert(reinterpret_cast<uintptr_t>(file_buffer.get()));
}
return file_buffer;
2021-11-05 11:55:30 +00:00
}
void RemoteCacheController::deallocFile(std::unique_ptr<ReadBufferFromFileBase> file_buffer)
2021-11-05 11:55:30 +00:00
{
if (!file_buffer)
2021-11-05 11:55:30 +00:00
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Try to release a null file buffer for ", local_path.string());
}
auto buffer_ref = reinterpret_cast<uintptr_t>(file_buffer.get());
std::lock_guard lock{mutex};
auto it = opened_file_buffer_refs.find(buffer_ref);
if (it == opened_file_buffer_refs.end())
{
throw Exception(
2021-11-29 12:19:36 +00:00
ErrorCodes::BAD_ARGUMENTS,
"Try to deallocate file with invalid handler remote path: {}, local path: {}",
2021-12-06 10:19:36 +00:00
file_metadata_ptr->getRemotePath(),
2021-11-30 04:05:18 +00:00
local_path.string());
2021-11-05 11:55:30 +00:00
}
opened_file_buffer_refs.erase(it);
2021-11-05 11:55:30 +00:00
}
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()
{
if (file_cache_controller)
file_cache_controller->deallocFile(std::move(file_buffer));
}
2021-11-05 11:55:30 +00:00
2021-12-06 10:19:36 +00:00
std::unique_ptr<ReadBuffer> RemoteReadBuffer::create(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, 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-12-06 10:19:36 +00:00
auto remote_path = remote_file_metadata->getRemotePath();
2021-11-29 05:01:03 +00:00
auto remote_read_buffer = std::make_unique<RemoteReadBuffer>(buff_size);
RemoteReadBufferCacheError error;
2021-11-05 11:55:30 +00:00
2021-12-06 10:19:36 +00:00
std::tie(remote_read_buffer->file_cache_controller, read_buffer, error) = RemoteReadBufferCache::instance().createReader(context, remote_file_metadata, read_buffer);
if (remote_read_buffer->file_cache_controller == nullptr)
2021-11-05 11:55:30 +00:00
{
LOG_ERROR(log, "Failed to allocate local file for remote path: {}, reason: {}.", remote_path, error);
// read_buffer is the input one.
return read_buffer;
}
else
{
remote_read_buffer->file_buffer = remote_read_buffer->file_cache_controller->allocFile();
if (!remote_read_buffer->file_buffer)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Create file readbuffer failed. {}",
remote_read_buffer->file_cache_controller->getLocalPath().string());
2021-11-05 11:55:30 +00:00
}
2021-12-06 10:19:36 +00:00
remote_read_buffer->remote_file_size = remote_file_metadata->getFileSize();
2021-11-29 05:01:03 +00:00
return remote_read_buffer;
2021-11-05 11:55:30 +00:00
}
bool RemoteReadBuffer::nextImpl()
{
auto start_offset = file_buffer->getPosition();
auto end_offset = start_offset + file_buffer->internalBuffer().size();
file_cache_controller->waitMoreData(start_offset, end_offset);
auto status = file_buffer->next();
if (status)
BufferBase::set(file_buffer->buffer().begin(),
file_buffer->buffer().size(),
file_buffer->offset());
return status;
2021-11-05 11:55:30 +00:00
}
off_t RemoteReadBuffer::seek(off_t offset, int whence)
{
if (!file_buffer)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot call seek() in this buffer. It's a bug!");
/*
2021-12-02 10:19:15 +00:00
* Need to wait here. For example, the current file has been download at position X, but here we try to seek to
* postition Y (Y > X), it would fail.
*/
file_cache_controller->waitMoreData(offset, offset + file_buffer->internalBuffer().size());
auto ret = file_buffer->seek(offset, whence);
BufferBase::set(file_buffer->buffer().begin(),
file_buffer->buffer().size(),
file_buffer->offset());
return ret;
2021-11-05 11:55:30 +00:00
}
off_t RemoteReadBuffer::getPosition()
{
return file_buffer->getPosition();
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-12-06 10:19:36 +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)
{
std::vector<fs::path> invalid_pathes;
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)
{
invalid_pathes.emplace_back(path);
continue;
}
if (!lru_caches->set(path, cache_controller))
{
invalid_pathes.emplace_back(path);
}
}
for (auto & path : invalid_pathes)
{
fs::remove_all(path);
}
return;
}
2021-11-29 08:22:43 +00:00
for (auto const & dir : fs::directory_iterator{current_path})
{
2021-12-06 10:19:36 +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);
2021-12-06 10:19:36 +00:00
recoverCachedFilesMetadata(root_dir, 1, 2);
2021-11-30 04:05:18 +00:00
initialized = true;
LOG_INFO(log, "Recovered from directory:{}", root_dir);
2021-11-30 04:05:18 +00:00
}
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-11 11:50:08 +00:00
local_cache_bytes_read_before_flush = bytes_read_before_flush_;
lru_caches = std::make_unique<CacheType>(limit_size_);
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-12-06 10:19:36 +00:00
String RemoteReadBufferCache::calculateLocalPath(IRemoteFileMetadataPtr metadata) const
2021-11-05 11:55:30 +00:00
{
// add version into the full_path, and not block to read the new version
2021-12-06 10:19:36 +00:00
String full_path = metadata->getName() + ":" + metadata->getRemotePath()
+ ":" + metadata->getVersion();
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
}
std::tuple<RemoteCacheControllerPtr, std::unique_ptr<ReadBuffer>, RemoteReadBufferCacheError>
2021-12-06 10:19:36 +00:00
RemoteReadBufferCache::createReader(ContextPtr context, IRemoteFileMetadataPtr remote_file_metadata, std::unique_ptr<ReadBuffer> & read_buffer)
2021-11-05 11:55:30 +00:00
{
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, std::move(read_buffer), RemoteReadBufferCacheError::NOT_INIT};
2021-11-05 11:55:30 +00:00
}
2021-12-06 10:19:36 +00:00
auto remote_path = remote_file_metadata->getRemotePath();
const auto & last_modification_timestamp = remote_file_metadata->getLastModificationTimestamp();
auto local_path = calculateLocalPath(remote_file_metadata);
2021-11-05 11:55:30 +00:00
std::lock_guard lock(mutex);
auto cache = lru_caches->get(local_path);
if (cache)
2021-11-05 11:55:30 +00:00
{
// the remote file has been updated, need to redownload
2021-12-06 10:19:36 +00:00
if (!cache->isValid() || cache->checkFileChanged(remote_file_metadata))
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->getLastModificationTimestamp()),
2021-11-29 06:50:33 +00:00
std::to_string(last_modification_timestamp));
cache->markInvalid();
2021-11-05 11:55:30 +00:00
}
else
{
return {cache, nullptr, RemoteReadBufferCacheError::OK};
2021-11-05 11:55:30 +00:00
}
}
if (!fs::exists(local_path))
fs::create_directories(local_path);
2021-11-05 11:55:30 +00:00
// cache is not found or is invalid
2021-12-06 10:19:36 +00:00
auto new_cache = std::make_shared<RemoteCacheController>(remote_file_metadata, local_path, local_cache_bytes_read_before_flush);
if (!lru_caches->set(local_path, new_cache))
2021-11-05 11:55:30 +00:00
{
LOG_ERROR(log, "Insert the new cache failed. new file size:{}, current total size:{}",
2021-12-06 10:19:36 +00:00
remote_file_metadata->getFileSize(),
lru_caches->weight());
return {nullptr, std::move(read_buffer), RemoteReadBufferCacheError::DISK_FULL};
2021-11-05 11:55:30 +00:00
}
new_cache->startBackgroundDownload(std::move(read_buffer), context->getSchedulePool());
return {new_cache, nullptr, RemoteReadBufferCacheError::OK};
2021-11-05 11:55:30 +00:00
}
}