ClickHouse/src/Storages/StorageMemory.h

129 lines
4.6 KiB
C++
Raw Normal View History

2011-10-31 17:55:06 +00:00
#pragma once
#include <atomic>
#include <optional>
#include <mutex>
2021-10-02 07:13:14 +00:00
#include <base/shared_ptr_helper.h>
#include <Core/NamesAndTypes.h>
#include <Storages/IStorage.h>
2011-10-31 17:55:06 +00:00
2020-10-04 16:28:36 +00:00
#include <Common/MultiVersion.h>
2011-10-31 17:55:06 +00:00
namespace DB
{
2017-04-16 15:00:33 +00:00
/** Implements storage in the RAM.
* Suitable for temporary data.
* It does not support keys.
* Data is stored as a set of blocks and is not stored anywhere else.
2011-10-31 17:55:06 +00:00
*/
2021-06-15 19:55:21 +00:00
class StorageMemory final : public shared_ptr_helper<StorageMemory>, public IStorage
2011-10-31 17:55:06 +00:00
{
2021-07-23 19:33:59 +00:00
friend class MemorySink;
2021-06-15 19:55:21 +00:00
friend struct shared_ptr_helper<StorageMemory>;
2011-10-31 17:55:06 +00:00
public:
String getName() const override { return "Memory"; }
2011-10-31 17:55:06 +00:00
2020-10-04 16:28:36 +00:00
size_t getSize() const { return data.get()->size(); }
struct SnapshotData : public StorageSnapshot::Data
{
std::shared_ptr<const Blocks> blocks;
};
StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot) const override;
2020-08-03 13:54:14 +00:00
Pipe read(
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
2017-06-02 15:54:39 +00:00
unsigned num_streams) override;
2011-10-31 17:55:06 +00:00
bool supportsParallelInsert() const override { return true; }
2020-12-22 16:40:53 +00:00
bool supportsSubcolumns() const override { return true; }
/// Smaller blocks (e.g. 64K rows) are better for CPU cache.
bool prefersLargeBlocks() const override { return false; }
bool hasEvenlyDistributedRead() const override { return true; }
2021-07-23 19:33:59 +00:00
SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override;
2011-10-31 17:55:06 +00:00
2020-01-22 11:30:11 +00:00
void drop() override;
2018-04-21 00:35:20 +00:00
void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override;
void mutate(const MutationCommands & commands, ContextPtr context) override;
2020-09-22 09:23:46 +00:00
void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override;
2018-04-21 00:35:20 +00:00
2020-11-25 13:47:32 +00:00
std::optional<UInt64> totalRows(const Settings &) const override;
std::optional<UInt64> totalBytes(const Settings &) const override;
2020-09-04 08:36:47 +00:00
/** Delays initialization of StorageMemory::read() until the first read is actually happen.
* Usually, fore code like this:
*
* auto out = StorageMemory::write();
* auto in = StorageMemory::read();
* out->write(new_data);
*
* `new_data` won't appear into `in`.
* However, if delayReadForGlobalSubqueries is called, first read from `in` will check for new_data and return it.
*
*
* Why is delayReadForGlobalSubqueries needed?
*
* The fact is that when processing a query of the form
* SELECT ... FROM remote_test WHERE column GLOBAL IN (subquery),
* if the distributed remote_test table contains localhost as one of the servers,
* the query will be interpreted locally again (and not sent over TCP, as in the case of a remote server).
*
* The query execution pipeline will be:
* CreatingSets
* subquery execution, filling the temporary table with _data1 (1)
* CreatingSets
* reading from the table _data1, creating the set (2)
* read from the table subordinate to remote_test.
*
* (The second part of the pipeline under CreateSets is a reinterpretation of the query inside StorageDistributed,
* the query differs in that the database name and tables are replaced with subordinates, and the subquery is replaced with _data1.)
*
* But when creating the pipeline, when creating the source (2), it will be found that the _data1 table is empty
* (because the query has not started yet), and empty source will be returned as the source.
* And then, when the query is executed, an empty set will be created in step (2).
*
* Therefore, we make the initialization of step (2) delayed
* - so that it does not occur until step (1) is completed, on which the table will be populated.
*/
void delayReadForGlobalSubqueries() { delay_read_for_global_subqueries = true; }
2011-10-31 17:55:06 +00:00
private:
2021-04-06 10:41:48 +00:00
/// MultiVersion data storage, so that we can copy the vector of blocks to readers.
2021-02-07 01:41:31 +00:00
2021-02-12 00:25:00 +00:00
MultiVersion<Blocks> data;
mutable std::mutex mutex;
2020-09-04 08:36:47 +00:00
bool delay_read_for_global_subqueries = false;
std::atomic<size_t> total_size_bytes = 0;
std::atomic<size_t> total_size_rows = 0;
2021-02-07 01:41:31 +00:00
bool compress;
protected:
2021-02-07 01:41:31 +00:00
StorageMemory(
const StorageID & table_id_,
ColumnsDescription columns_description_,
ConstraintsDescription constraints_,
2021-04-23 12:18:23 +00:00
const String & comment,
2021-02-07 01:41:31 +00:00
bool compress_ = false);
2011-10-31 17:55:06 +00:00
};
}