ClickHouse/src/Storages/System/StorageSystemZeros.h

46 lines
1.4 KiB
C++
Raw Normal View History

2020-03-10 13:01:29 +00:00
#pragma once
2021-10-02 07:13:14 +00:00
#include <base/shared_ptr_helper.h>
2020-03-10 13:01:29 +00:00
#include <optional>
#include <Storages/IStorage.h>
namespace DB
{
/** Implements a table engine for the system table "zeros".
* The table contains the only column zero UInt8.
* From this table, you can read non-materialized zeros.
*
* You could also specify a limit (how many zeros to give).
* If multithreaded is specified, zeros will be generated in several streams.
*/
2021-06-15 19:55:21 +00:00
class StorageSystemZeros final : public shared_ptr_helper<StorageSystemZeros>, public IStorage
2020-03-10 13:01:29 +00:00
{
2021-06-15 19:55:21 +00:00
friend struct shared_ptr_helper<StorageSystemZeros>;
2020-03-10 13:01:29 +00:00
public:
std::string getName() const override { return "SystemZeros"; }
2020-08-06 12:24:05 +00:00
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
unsigned num_streams) override;
2020-03-10 13:01:29 +00:00
bool hasEvenlyDistributedRead() const override { return true; }
bool isSystemStorage() const override { return true; }
2020-03-10 13:01:29 +00:00
private:
bool multithreaded;
std::optional<UInt64> limit;
protected:
/// If even_distribution is true, numbers are distributed evenly between streams.
/// Otherwise, streams concurrently increment atomic.
StorageSystemZeros(const StorageID & table_id_, bool multithreaded_, std::optional<UInt64> limit_ = std::nullopt);
2020-03-10 13:01:29 +00:00
};
}