2020-04-14 19:47:19 +00:00
|
|
|
#pragma once
|
|
|
|
|
2021-10-15 20:18:20 +00:00
|
|
|
#include <Formats/NativeReader.h>
|
|
|
|
#include <Formats/NativeWriter.h>
|
2020-09-01 22:25:10 +00:00
|
|
|
#include <Core/BackgroundSchedulePool.h>
|
2020-04-14 19:47:19 +00:00
|
|
|
#include <Disks/IDisk.h>
|
2020-10-29 16:18:25 +00:00
|
|
|
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
|
|
|
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
|
2020-04-14 19:47:19 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
class MergeTreeData;
|
|
|
|
|
2020-06-22 18:56:53 +00:00
|
|
|
/** WAL stores addditions and removals of data parts in in-memory format.
|
|
|
|
* Format of data in WAL:
|
|
|
|
* - version
|
|
|
|
* - type of action (ADD or DROP)
|
|
|
|
* - part name
|
|
|
|
* - part's block in Native format. (for ADD action)
|
|
|
|
*/
|
2020-04-14 19:47:19 +00:00
|
|
|
class MergeTreeWriteAheadLog
|
|
|
|
{
|
|
|
|
public:
|
2020-05-29 15:02:12 +00:00
|
|
|
/// Append-only enum. It is serialized to WAL
|
|
|
|
enum class ActionType : UInt8
|
|
|
|
{
|
|
|
|
ADD_PART = 0,
|
|
|
|
DROP_PART = 1,
|
|
|
|
};
|
|
|
|
|
2020-10-16 16:48:59 +00:00
|
|
|
struct ActionMetadata
|
|
|
|
{
|
|
|
|
/// The minimum version of WAL reader that can understand metadata written by current ClickHouse version.
|
|
|
|
/// This field must be increased when making backwards incompatible changes.
|
|
|
|
///
|
|
|
|
/// The same approach can be used recursively inside metadata.
|
|
|
|
UInt8 min_compatible_version = 0;
|
|
|
|
|
2020-10-29 16:18:25 +00:00
|
|
|
/// Actual metadata.
|
|
|
|
UUID part_uuid = UUIDHelpers::Nil;
|
|
|
|
|
2020-10-16 16:48:59 +00:00
|
|
|
void write(WriteBuffer & meta_out) const;
|
|
|
|
void read(ReadBuffer & meta_in);
|
2020-11-02 22:36:32 +00:00
|
|
|
|
|
|
|
private:
|
|
|
|
static constexpr auto JSON_KEY_PART_UUID = "part_uuid";
|
|
|
|
|
|
|
|
String toJSON() const;
|
|
|
|
void fromJSON(const String & buf);
|
2020-10-16 16:48:59 +00:00
|
|
|
};
|
|
|
|
|
2020-10-29 16:18:25 +00:00
|
|
|
constexpr static UInt8 WAL_VERSION = 1;
|
2020-04-14 19:47:19 +00:00
|
|
|
constexpr static auto WAL_FILE_NAME = "wal";
|
|
|
|
constexpr static auto WAL_FILE_EXTENSION = ".bin";
|
2020-06-22 18:56:53 +00:00
|
|
|
constexpr static auto DEFAULT_WAL_FILE_NAME = "wal.bin";
|
2020-04-14 19:47:19 +00:00
|
|
|
|
2020-09-01 22:25:10 +00:00
|
|
|
MergeTreeWriteAheadLog(MergeTreeData & storage_, const DiskPtr & disk_,
|
2020-06-22 18:56:53 +00:00
|
|
|
const String & name = DEFAULT_WAL_FILE_NAME);
|
2020-04-14 19:47:19 +00:00
|
|
|
|
2020-09-11 13:09:26 +00:00
|
|
|
~MergeTreeWriteAheadLog();
|
|
|
|
|
2020-10-29 16:18:25 +00:00
|
|
|
void addPart(DataPartInMemoryPtr & part);
|
2020-05-29 15:02:12 +00:00
|
|
|
void dropPart(const String & part_name);
|
2021-05-21 16:14:01 +00:00
|
|
|
std::vector<MergeTreeMutableDataPartPtr> restore(const StorageMetadataPtr & metadata_snapshot, ContextPtr context);
|
2020-04-14 19:47:19 +00:00
|
|
|
|
2020-05-27 20:05:55 +00:00
|
|
|
using MinMaxBlockNumber = std::pair<Int64, Int64>;
|
|
|
|
static std::optional<MinMaxBlockNumber> tryParseMinMaxBlockNumber(const String & filename);
|
2022-06-04 23:05:15 +00:00
|
|
|
void shutdown();
|
2020-05-27 20:05:55 +00:00
|
|
|
|
2020-04-14 19:47:19 +00:00
|
|
|
private:
|
2020-04-30 15:27:39 +00:00
|
|
|
void init();
|
2020-09-10 23:24:16 +00:00
|
|
|
void rotate(const std::unique_lock<std::mutex> & lock);
|
|
|
|
void sync(std::unique_lock<std::mutex> & lock);
|
2020-04-14 19:47:19 +00:00
|
|
|
|
|
|
|
const MergeTreeData & storage;
|
|
|
|
DiskPtr disk;
|
2020-05-14 20:08:15 +00:00
|
|
|
String name;
|
2020-04-14 19:47:19 +00:00
|
|
|
String path;
|
|
|
|
|
|
|
|
std::unique_ptr<WriteBuffer> out;
|
2021-10-08 17:21:19 +00:00
|
|
|
std::unique_ptr<NativeWriter> block_out;
|
2020-04-14 19:47:19 +00:00
|
|
|
|
|
|
|
Int64 min_block_number = std::numeric_limits<Int64>::max();
|
2020-05-14 20:08:15 +00:00
|
|
|
Int64 max_block_number = -1;
|
2020-04-14 19:47:19 +00:00
|
|
|
|
2020-09-01 22:25:10 +00:00
|
|
|
BackgroundSchedulePool & pool;
|
|
|
|
BackgroundSchedulePoolTaskHolder sync_task;
|
2020-09-10 23:24:16 +00:00
|
|
|
std::condition_variable sync_cv;
|
2020-09-01 22:25:10 +00:00
|
|
|
|
|
|
|
size_t bytes_at_last_sync = 0;
|
|
|
|
bool sync_scheduled = false;
|
2022-06-04 23:05:15 +00:00
|
|
|
bool shutted_down = false;
|
2020-09-01 22:25:10 +00:00
|
|
|
|
2020-04-14 19:47:19 +00:00
|
|
|
mutable std::mutex write_mutex;
|
2021-08-26 11:01:15 +00:00
|
|
|
|
|
|
|
Poco::Logger * log;
|
2020-04-14 19:47:19 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|