ClickHouse/src/Storages/FileLog/StorageFileLog.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

205 lines
5.5 KiB
C++
Raw Normal View History

2021-06-09 02:03:36 +00:00
#pragma once
#include <Storages/FileLog/Buffer_fwd.h>
2021-10-18 06:38:47 +00:00
#include <Storages/FileLog/FileLogDirectoryWatcher.h>
2021-09-04 17:04:35 +00:00
#include <Storages/FileLog/FileLogSettings.h>
2021-06-09 02:03:36 +00:00
#include <Core/BackgroundSchedulePool.h>
#include <Storages/IStorage.h>
#include <Common/SettingsChanges.h>
#include <atomic>
2021-10-17 14:49:27 +00:00
#include <condition_variable>
2021-10-04 12:33:05 +00:00
#include <filesystem>
2021-09-04 17:04:35 +00:00
#include <fstream>
2021-10-04 12:33:05 +00:00
#include <mutex>
2021-10-07 12:15:36 +00:00
#include <optional>
2021-06-09 02:03:36 +00:00
namespace DB
{
2021-09-30 16:02:17 +00:00
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
2021-10-17 14:49:27 +00:00
class FileLogDirectoryWatcher;
class StorageFileLog final : public IStorage, WithContext
2021-06-09 02:03:36 +00:00
{
public:
StorageFileLog(
const StorageID & table_id_,
ContextPtr context_,
const ColumnsDescription & columns_,
const String & path_,
const String & metadata_base_path_,
const String & format_name_,
std::unique_ptr<FileLogSettings> settings,
const String & comment,
bool attach);
2021-09-05 06:32:32 +00:00
2021-06-09 02:03:36 +00:00
using Files = std::vector<String>;
std::string getName() const override { return "FileLog"; }
bool noPushingToViews() const override { return true; }
void startup() override;
void shutdown() override;
Pipe read(
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
2021-06-09 02:03:36 +00:00
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
size_t num_streams) override;
2021-06-09 02:03:36 +00:00
2021-09-26 07:22:45 +00:00
void drop() override;
2021-06-09 02:03:36 +00:00
const auto & getFormatName() const { return format_name; }
2021-09-26 07:22:45 +00:00
enum class FileStatus
{
2022-02-28 00:15:37 +00:00
OPEN, /// First time open file after table start up.
2021-09-26 07:22:45 +00:00
NO_CHANGE,
UPDATED,
REMOVED,
};
struct FileContext
{
FileStatus status = FileStatus::OPEN;
UInt64 inode{};
2021-10-07 12:15:36 +00:00
std::optional<std::ifstream> reader = std::nullopt;
2021-09-26 07:22:45 +00:00
};
struct FileMeta
{
String file_name;
2021-10-17 14:49:27 +00:00
UInt64 last_writen_position = 0;
UInt64 last_open_end = 0;
2021-09-26 07:22:45 +00:00
};
using InodeToFileMeta = std::unordered_map<UInt64, FileMeta>;
using FileNameToContext = std::unordered_map<String, FileContext>;
struct FileInfos
{
InodeToFileMeta meta_by_inode;
FileNameToContext context_by_name;
2022-02-28 00:15:37 +00:00
/// File names without path.
2021-09-26 07:22:45 +00:00
Names file_names;
};
auto & getFileInfos() { return file_infos; }
2022-02-25 15:35:37 +00:00
String getFullMetaPath(const String & file_name) const { return std::filesystem::path(metadata_base_path) / file_name; }
2021-09-30 16:02:17 +00:00
String getFullDataPath(const String & file_name) const { return std::filesystem::path(root_data_path) / file_name; }
2021-06-09 02:03:36 +00:00
2021-09-24 16:44:22 +00:00
NamesAndTypesList getVirtuals() const override;
static Names getVirtualColumnNames();
2021-09-26 07:22:45 +00:00
static UInt64 getInode(const String & file_name);
2021-09-27 04:39:50 +00:00
void openFilesAndSetPos();
2021-10-06 08:08:49 +00:00
2021-10-06 15:48:23 +00:00
/// Used in FileLogSource when finish generating all blocks.
/// Each stream responsible for close its files and store meta.
2021-10-06 10:37:58 +00:00
void closeFilesAndStoreMeta(size_t start, size_t end);
2021-10-06 08:08:49 +00:00
/// Used in FileLogSource after generating every block
2021-10-06 10:37:58 +00:00
void storeMetas(size_t start, size_t end);
2021-09-30 16:02:17 +00:00
static void assertStreamGood(const std::ifstream & reader);
template <typename K, typename V>
static V & findInMap(std::unordered_map<K, V> & map, const K & key)
{
if (auto it = map.find(key); it != map.end())
return it->second;
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "The key {} doesn't exist.", key);
}
2021-09-27 04:39:50 +00:00
2021-10-07 05:56:09 +00:00
void increaseStreams();
void reduceStreams();
2021-10-18 06:38:47 +00:00
void wakeUp();
2021-10-17 14:49:27 +00:00
const auto & getFileLogSettings() const { return filelog_settings; }
2021-06-09 02:03:36 +00:00
private:
2021-09-04 17:04:35 +00:00
std::unique_ptr<FileLogSettings> filelog_settings;
2021-09-26 07:22:45 +00:00
const String path;
bool path_is_directory = true;
2021-06-09 02:03:36 +00:00
2021-09-26 07:22:45 +00:00
/// If path argument of the table is a regular file, it equals to user_files_path
/// otherwise, it equals to user_files_path/ + path_argument/, e.g. path
String root_data_path;
2022-02-25 15:35:37 +00:00
String metadata_base_path;
2021-09-04 17:04:35 +00:00
2021-09-26 07:22:45 +00:00
FileInfos file_infos;
2021-09-04 17:04:35 +00:00
2021-09-26 07:22:45 +00:00
const String format_name;
Poco::Logger * log;
2021-09-04 17:04:35 +00:00
2021-09-21 16:11:35 +00:00
uint64_t milliseconds_to_wait;
2021-10-07 05:56:09 +00:00
/// In order to avoid data race, using a naive trick to forbid execute two select
/// simultaneously, although read is not useful in this engine. Using an atomic
/// variable to records current unfinishing streams, then if have unfinishing streams,
/// later select should forbid to execute.
std::atomic<int> running_streams = 0;
2021-10-17 14:49:27 +00:00
std::mutex mutex;
bool has_new_events = false;
std::condition_variable cv;
2021-11-29 13:13:20 +00:00
std::atomic<bool> mv_attached = false;
2021-10-18 15:44:54 +00:00
std::mutex file_infos_mutex;
2021-10-18 06:38:47 +00:00
struct TaskContext
{
BackgroundSchedulePool::TaskHolder holder;
std::atomic<bool> stream_cancelled {false};
explicit TaskContext(BackgroundSchedulePool::TaskHolder&& task_) : holder(std::move(task_))
{
}
};
std::shared_ptr<TaskContext> task;
std::unique_ptr<FileLogDirectoryWatcher> directory_watch = nullptr;
2021-09-26 07:22:45 +00:00
void loadFiles();
void loadMetaFiles(bool attach);
2021-09-22 02:07:59 +00:00
2021-06-09 02:03:36 +00:00
void threadFunc();
size_t getPollMaxBatchSize() const;
size_t getMaxBlockSize() const;
size_t getPollTimeoutMillisecond() const;
bool streamToViews();
bool checkDependencies(const StorageID & table_id);
2021-09-04 17:04:35 +00:00
2021-09-26 07:22:45 +00:00
bool updateFileInfos();
2021-10-18 07:27:37 +00:00
size_t getTableDependentCount() const;
2021-09-30 16:02:17 +00:00
/// Used in shutdown()
void serialize() const;
2022-02-28 00:15:37 +00:00
/// Used in FileSource closeFileAndStoreMeta(file_name).
2021-09-30 16:02:17 +00:00
void serialize(UInt64 inode, const FileMeta & file_meta) const;
2021-09-26 07:22:45 +00:00
void deserialize();
2021-10-06 10:37:58 +00:00
static void checkOffsetIsValid(const String & full_name, UInt64 offset);
2021-06-09 02:03:36 +00:00
};
}