ClickHouse/src/Interpreters/TemporaryDataOnDisk.h

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

217 lines
6.4 KiB
C++
Raw Normal View History

#pragma once
2024-04-28 11:20:59 +00:00
#include <atomic>
2024-05-07 20:43:40 +00:00
#include <mutex>
#include <boost/noncopyable.hpp>
2024-04-28 11:20:59 +00:00
#include <IO/ReadBufferFromFile.h>
#include <Compression/CompressedReadBuffer.h>
#include <Formats/NativeReader.h>
#include <Core/Block.h>
#include <Disks/IVolume.h>
#include <Disks/TemporaryFileOnDisk.h>
2022-12-06 17:27:05 +00:00
#include <Interpreters/Cache/FileSegment.h>
#include <Common/CurrentMetrics.h>
2022-10-05 16:35:10 +00:00
namespace CurrentMetrics
{
extern const Metric TemporaryFilesUnknown;
}
namespace DB
{
class TemporaryDataOnDiskScope;
using TemporaryDataOnDiskScopePtr = std::shared_ptr<TemporaryDataOnDiskScope>;
class TemporaryDataOnDisk;
2022-09-21 12:51:46 +00:00
using TemporaryDataOnDiskPtr = std::unique_ptr<TemporaryDataOnDisk>;
class TemporaryFileStream;
2022-09-21 12:51:46 +00:00
using TemporaryFileStreamPtr = std::unique_ptr<TemporaryFileStream>;
2024-01-24 17:26:11 +00:00
class FileCache;
struct TemporaryDataOnDiskSettings
{
/// Max size on disk, if 0 there will be no limit
size_t max_size_on_disk = 0;
/// Compression codec for temporary data, if empty no compression will be used. LZ4 by default
String compression_codec = "LZ4";
};
/*
2022-09-21 12:51:46 +00:00
* Used to account amount of temporary data written to disk.
* If limit is set, throws exception if limit is exceeded.
2022-09-16 11:00:28 +00:00
* Data can be nested, so parent scope accounts all data written by children.
* Scopes are: global -> per-user -> per-query -> per-purpose (sorting, aggregation, etc).
*/
class TemporaryDataOnDiskScope : boost::noncopyable
{
public:
struct StatAtomic
{
std::atomic<size_t> compressed_size;
std::atomic<size_t> uncompressed_size;
};
explicit TemporaryDataOnDiskScope(VolumePtr volume_, TemporaryDataOnDiskSettings settings_)
: volume(std::move(volume_))
, settings(std::move(settings_))
2022-12-06 17:27:05 +00:00
{}
explicit TemporaryDataOnDiskScope(VolumePtr volume_, FileCache * file_cache_, TemporaryDataOnDiskSettings settings_)
: volume(std::move(volume_))
, file_cache(file_cache_)
, settings(std::move(settings_))
2022-12-06 17:27:05 +00:00
{}
explicit TemporaryDataOnDiskScope(TemporaryDataOnDiskScopePtr parent_, TemporaryDataOnDiskSettings settings_)
: parent(std::move(parent_))
, volume(parent->volume)
, file_cache(parent->file_cache)
, settings(std::move(settings_))
2022-12-06 17:27:05 +00:00
{}
/// TODO: remove
/// Refactor all code that uses volume directly to use TemporaryDataOnDisk.
2022-12-06 17:27:05 +00:00
VolumePtr getVolume() const { return volume; }
const TemporaryDataOnDiskSettings & getSettings() const { return settings; }
protected:
void deltaAllocAndCheck(ssize_t compressed_delta, ssize_t uncompressed_delta);
TemporaryDataOnDiskScopePtr parent = nullptr;
2022-12-06 10:04:15 +00:00
2022-12-06 17:27:05 +00:00
VolumePtr volume = nullptr;
FileCache * file_cache = nullptr;
StatAtomic stat;
const TemporaryDataOnDiskSettings settings;
};
/*
* Holds the set of temporary files.
* New file stream is created with `createStream`.
* Streams are owned by this object and will be deleted when it is deleted.
* It's a leaf node in temporary data scope tree.
*/
class TemporaryDataOnDisk : private TemporaryDataOnDiskScope
{
2022-09-21 12:51:46 +00:00
friend class TemporaryFileStream; /// to allow it to call `deltaAllocAndCheck` to account data
2022-10-05 16:35:10 +00:00
public:
using TemporaryDataOnDiskScope::StatAtomic;
explicit TemporaryDataOnDisk(TemporaryDataOnDiskScopePtr parent_);
2022-10-05 16:35:10 +00:00
explicit TemporaryDataOnDisk(TemporaryDataOnDiskScopePtr parent_, CurrentMetrics::Metric metric_scope);
2022-09-28 10:22:16 +00:00
/// If max_file_size > 0, then check that there's enough space on the disk and throw an exception in case of lack of free space
2022-10-05 16:35:10 +00:00
TemporaryFileStream & createStream(const Block & header, size_t max_file_size = 0);
/// Write raw data directly into buffer.
/// Differences from `createStream`:
/// 1) it doesn't account data in parent scope
/// 2) returned buffer owns resources (instead of TemporaryDataOnDisk itself)
/// If max_file_size > 0, then check that there's enough space on the disk and throw an exception in case of lack of free space
2023-11-27 19:39:08 +00:00
std::unique_ptr<WriteBufferFromFileBase> createRawStream(size_t max_file_size = 0);
2022-09-28 10:22:16 +00:00
std::vector<TemporaryFileStream *> getStreams() const;
2022-09-21 12:51:46 +00:00
bool empty() const;
const StatAtomic & getStat() const { return stat; }
private:
FileSegmentsHolderPtr createCacheFile(size_t max_file_size);
TemporaryFileOnDiskHolder createRegularFile(size_t max_file_size);
2022-12-06 17:27:05 +00:00
2022-09-21 12:51:46 +00:00
mutable std::mutex mutex;
std::vector<TemporaryFileStreamPtr> streams TSA_GUARDED_BY(mutex);
2022-10-05 16:35:10 +00:00
typename CurrentMetrics::Metric current_metric_scope = CurrentMetrics::TemporaryFilesUnknown;
};
/*
* Data can be written into this stream and then read.
* After finish writing, call `finishWriting` and then either call `read` or 'getReadStream'(only one of the two) to read the data.
* Account amount of data written to disk in parent scope.
*/
2022-09-01 12:22:49 +00:00
class TemporaryFileStream : boost::noncopyable
{
public:
2024-05-07 20:43:40 +00:00
struct Reader
{
Reader(const String & path, const Block & header_, size_t size = 0);
explicit Reader(const String & path, size_t size = 0);
Block read();
ReadBufferFromFile in_file_buf;
CompressedReadBuffer in_compressed_buf;
NativeReader in_reader;
};
struct Stat
{
/// Statistics for file
/// Non-atomic because we don't allow to `read` or `write` into single file from multiple threads
size_t compressed_size = 0;
size_t uncompressed_size = 0;
2022-10-05 16:35:10 +00:00
size_t num_rows = 0;
};
2022-12-06 17:27:05 +00:00
TemporaryFileStream(TemporaryFileOnDiskHolder file_, const Block & header_, TemporaryDataOnDisk * parent_);
2023-01-04 19:57:42 +00:00
TemporaryFileStream(FileSegmentsHolderPtr segments_, const Block & header_, TemporaryDataOnDisk * parent_);
2022-12-06 10:04:15 +00:00
size_t write(const Block & block);
2022-12-06 17:27:05 +00:00
void flush();
Stat finishWriting();
2024-04-28 11:20:59 +00:00
Stat finishWritingAsyncSafe();
bool isWriteFinished() const;
2024-05-07 20:43:40 +00:00
std::unique_ptr<Reader> getReadStream();
2024-04-28 11:20:59 +00:00
Block read();
2022-12-06 17:27:05 +00:00
String getPath() const;
size_t getSize() const;
2022-12-06 17:27:05 +00:00
Block getHeader() const { return header; }
2022-09-01 12:22:49 +00:00
2022-10-05 16:35:10 +00:00
/// Read finished and file released
bool isEof() const;
~TemporaryFileStream();
private:
2022-09-21 12:51:46 +00:00
void updateAllocAndCheck();
2022-09-01 12:22:49 +00:00
2022-10-05 16:35:10 +00:00
/// Release everything, close reader and writer, delete file
void release();
2022-09-01 12:22:49 +00:00
TemporaryDataOnDisk * parent;
Block header;
2022-09-01 12:22:49 +00:00
2022-12-06 17:27:05 +00:00
/// Data can be stored in file directly or in the cache
TemporaryFileOnDiskHolder file;
2023-01-04 19:57:42 +00:00
FileSegmentsHolderPtr segment_holder;
Stat stat;
2024-05-07 20:43:40 +00:00
std::once_flag finish_writing;
2024-04-28 11:20:59 +00:00
struct OutputWriter;
std::unique_ptr<OutputWriter> out_writer;
2024-05-07 20:43:40 +00:00
std::unique_ptr<Reader> in_reader;
};
}