Don't let storages inherit from boost::noncopyable

... IStorage has deleted copy ctor / assignment already
This commit is contained in:
Robert Schulze 2022-05-03 08:43:28 +02:00
parent 330212e0f4
commit 777b5bc15b
No known key found for this signature in database
GPG Key ID: 6501C40A34BE4E0E
116 changed files with 116 additions and 228 deletions

View File

@ -1,5 +1,6 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <base/types.h>
#include <memory>
@ -11,7 +12,7 @@ class ReadBufferFromFileBase;
class SeekableReadBuffer;
/// Interface for reading an archive.
class IArchiveReader : public std::enable_shared_from_this<IArchiveReader>
class IArchiveReader : public std::enable_shared_from_this<IArchiveReader>, boost::noncopyable
{
public:
virtual ~IArchiveReader() = default;

View File

@ -1,5 +1,6 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <base/types.h>
#include <memory>
@ -9,7 +10,7 @@ namespace DB
class WriteBufferFromFileBase;
/// Interface for writing an archive.
class IArchiveWriter : public std::enable_shared_from_this<IArchiveWriter>
class IArchiveWriter : public std::enable_shared_from_this<IArchiveWriter>, boost::noncopyable
{
public:
/// Destructors finalizes writing the archive.

View File

@ -3,7 +3,6 @@
#include <Common/config.h>
#if USE_MINIZIP
#include <boost/noncopyable.hpp>
#include <IO/Archives/IArchiveReader.h>
#include <mutex>
#include <vector>
@ -16,7 +15,7 @@ class ReadBufferFromFileBase;
class SeekableReadBuffer;
/// Implementation of IArchiveReader for reading zip archives.
class ZipArchiveReader : public IArchiveReader, boost::noncopyable
class ZipArchiveReader : public IArchiveReader
{
public:
/// Constructs an archive's reader that will read from a file in the local filesystem.

View File

@ -3,7 +3,6 @@
#include <Common/config.h>
#if USE_MINIZIP
#include <boost/noncopyable.hpp>
#include <IO/Archives/IArchiveWriter.h>
#include <mutex>
@ -14,7 +13,7 @@ class WriteBuffer;
class WriteBufferFromFileBase;
/// Implementation of IArchiveWriter for writing zip archives.
class ZipArchiveWriter : public IArchiveWriter, boost::noncopyable
class ZipArchiveWriter : public IArchiveWriter
{
public:
/// Constructs an archive that will be written as a file in the local filesystem.

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/FileLog/Buffer_fwd.h>
#include <Storages/FileLog/FileLogDirectoryWatcher.h>
#include <Storages/FileLog/FileLogSettings.h>
@ -25,7 +24,7 @@ namespace ErrorCodes
class FileLogDirectoryWatcher;
class StorageFileLog final : public IStorage, WithContext, boost::noncopyable
class StorageFileLog final : public IStorage, WithContext
{
public:
StorageFileLog(

View File

@ -4,7 +4,6 @@
#if USE_HDFS
#include <boost/noncopyable.hpp>
#include <Processors/Sources/SourceWithProgress.h>
#include <Storages/IStorage.h>
#include <Poco/URI.h>
@ -16,7 +15,7 @@ namespace DB
* This class represents table engine for external hdfs files.
* Read method is supported for now.
*/
class StorageHDFS final : public IStorage, WithContext, boost::noncopyable
class StorageHDFS final : public IStorage, WithContext
{
public:
StorageHDFS(

View File

@ -7,7 +7,6 @@
#include <memory>
#include <optional>
#include <boost/noncopyable.hpp>
#include <Client/Connection.h>
#include <Interpreters/Cluster.h>
#include <Storages/HDFS/StorageHDFS.h>
@ -17,7 +16,7 @@ namespace DB
class Context;
class StorageHDFSCluster : public IStorage, boost::noncopyable
class StorageHDFSCluster : public IStorage
{
public:
StorageHDFSCluster(

View File

@ -7,7 +7,6 @@
#include <Poco/URI.h>
#include <ThriftHiveMetastore.h>
#include <boost/noncopyable.hpp>
#include <Common/logger_useful.h>
#include <Interpreters/Context.h>
#include <Storages/IStorage.h>
@ -23,7 +22,7 @@ class HiveSettings;
* This class represents table engine for external hdfs files.
* Read method is supported for now.
*/
class StorageHive final : public IStorage, WithContext, boost::noncopyable
class StorageHive final : public IStorage, WithContext
{
public:
friend class StorageHiveSource;

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Core/BackgroundSchedulePool.h>
#include <Storages/IStorage.h>
#include <Storages/Kafka/Buffer_fwd.h>
@ -28,7 +27,7 @@ struct StorageKafkaInterceptors;
/** Implements a Kafka queue table engine that can be used as a persistent queue / buffer,
* or as a basic building block for creating pipelines with a continuous insertion / ETL.
*/
class StorageKafka final : public IStorage, WithContext, boost::noncopyable
class StorageKafka final : public IStorage, WithContext
{
friend struct StorageKafkaInterceptors;

View File

@ -11,7 +11,6 @@ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License. */
#include <boost/noncopyable.hpp>
#include <Storages/IStorage.h>
#include <Core/BackgroundSchedulePool.h>
@ -49,7 +48,7 @@ class Pipe;
using Pipes = std::vector<Pipe>;
class StorageLiveView final : public IStorage, WithContext, boost::noncopyable
class StorageLiveView final : public IStorage, WithContext
{
friend class LiveViewSource;
friend class LiveViewEventsSource;

View File

@ -3,7 +3,6 @@
#include <memory>
#include <utility>
#include <boost/noncopyable.hpp>
#include <Storages/MergeTree/IExecutableTask.h>
#include <Storages/MergeTree/MergeTask.h>
#include <Storages/MergeTree/ReplicatedMergeTreeQueue.h>
@ -15,7 +14,7 @@
namespace DB
{
class MergeFromLogEntryTask : public ReplicatedMergeMutateTaskBase, boost::noncopyable
class MergeFromLogEntryTask : public ReplicatedMergeMutateTaskBase
{
public:
template <class Callback>

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/MergeTree/IExecutableTask.h>
#include <Storages/MergeTree/MutateTask.h>
#include <Storages/MergeTree/ReplicatedMergeMutateTaskBase.h>
@ -11,7 +10,7 @@
namespace DB
{
class MutateFromLogEntryTask : public ReplicatedMergeMutateTaskBase, boost::noncopyable
class MutateFromLogEntryTask : public ReplicatedMergeMutateTaskBase
{
public:
template <typename Callback>

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/IStorage.h>
#include <Storages/MergeTree/IMergeTreeDataPart.h>
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
@ -15,7 +14,7 @@ namespace DB
{
/// A Storage that allows reading from a single MergeTree data part.
class StorageFromMergeTreeDataPart final : public IStorage, boost::noncopyable
class StorageFromMergeTreeDataPart final : public IStorage
{
public:
/// Used in part mutation.

View File

@ -5,7 +5,6 @@
#include <memory>
#include <unordered_map>
#include <boost/noncopyable.hpp>
#include <Parsers/ASTAlterQuery.h>
#include <Storages/IStorage_fwd.h>
#include <DataTypes/IDataType.h>

View File

@ -6,7 +6,6 @@
#include "PostgreSQLReplicationHandler.h"
#include "MaterializedPostgreSQLSettings.h"
#include <boost/noncopyable.hpp>
#include <Parsers/IAST.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTColumnDeclaration.h>
@ -61,7 +60,7 @@ namespace DB
*
**/
class StorageMaterializedPostgreSQL final : public IStorage, WithContext, boost::noncopyable
class StorageMaterializedPostgreSQL final : public IStorage, WithContext
{
public:
StorageMaterializedPostgreSQL(const StorageID & table_id_, ContextPtr context_,

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Core/BackgroundSchedulePool.h>
#include <Storages/IStorage.h>
#include <Poco/Semaphore.h>
@ -18,7 +17,7 @@
namespace DB
{
class StorageRabbitMQ final: public IStorage, WithContext, boost::noncopyable
class StorageRabbitMQ final: public IStorage, WithContext
{
public:
StorageRabbitMQ(

View File

@ -2,7 +2,6 @@
#include <memory>
#include <shared_mutex>
#include <boost/noncopyable.hpp>
#include <Storages/IStorage.h>
#include <rocksdb/status.h>
@ -19,7 +18,7 @@ namespace DB
class Context;
class StorageEmbeddedRocksDB final : public IStorage, WithContext, boost::noncopyable
class StorageEmbeddedRocksDB final : public IStorage, WithContext
{
friend class EmbeddedRocksDBSink;
public:

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -12,7 +11,7 @@ class Context;
/** Implements the `rocksdb` system table, which expose various rocksdb metrics.
*/
class StorageSystemRocksDB final : public IStorageSystemOneBlock<StorageSystemRocksDB>, boost::noncopyable
class StorageSystemRocksDB final : public IStorageSystemOneBlock<StorageSystemRocksDB>
{
public:
std::string getName() const override { return "SystemRocksDB"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Core/BackgroundSchedulePool.h>
#include <Core/NamesAndTypes.h>
#include <Storages/IStorage.h>
@ -41,7 +40,7 @@ namespace DB
* When you destroy a Buffer table, all remaining data is flushed to the subordinate table.
* The data in the buffer is not replicated, not logged to disk, not indexed. With a rough restart of the server, the data is lost.
*/
class StorageBuffer final : public IStorage, WithContext, boost::noncopyable
class StorageBuffer final : public IStorage, WithContext
{
friend class BufferSource;
friend class BufferSink;

View File

@ -2,7 +2,6 @@
#include <atomic>
#include <boost/noncopyable.hpp>
#include <Storages/IStorage.h>
#include <Interpreters/IExternalLoaderConfigRepository.h>
@ -12,7 +11,7 @@ namespace DB
struct DictionaryStructure;
class TableFunctionDictionary;
class StorageDictionary final : public IStorage, public WithContext, boost::noncopyable
class StorageDictionary final : public IStorage, public WithContext
{
friend class TableFunctionDictionary;

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/IStorage.h>
#include <Storages/Distributed/DirectoryMonitor.h>
#include <Storages/Distributed/DistributedSettings.h>
@ -35,7 +34,7 @@ using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
* You can pass one address, not several.
* In this case, the table can be considered remote, rather than distributed.
*/
class StorageDistributed final : public IStorage, WithContext, boost::noncopyable
class StorageDistributed final : public IStorage, WithContext
{
friend class DistributedSink;
friend class StorageDistributedDirectoryMonitor;

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Common/logger_useful.h>
#include <Storages/IStorage.h>
#include <Processors/Sources/ShellCommandSource.h>
@ -15,7 +14,7 @@ namespace DB
* Executable storage that will start process for read.
* ExecutablePool storage maintain pool of processes and take process from pool for read.
*/
class StorageExecutable final : public IStorage, boost::noncopyable
class StorageExecutable final : public IStorage
{
public:
StorageExecutable(

View File

@ -2,7 +2,6 @@
#include "config_core.h"
#include <boost/noncopyable.hpp>
#include <Storages/IStorage.h>
@ -16,7 +15,7 @@ struct ExternalDataSourceConfiguration;
/// A query to external database is passed to one replica on each shard, the result is united.
/// Replicas on each shard have the same priority, traversed replicas are moved to the end of the queue.
/// Similar approach is used for URL storage.
class StorageExternalDistributed final : public DB::IStorage, boost::noncopyable
class StorageExternalDistributed final : public DB::IStorage
{
public:
enum class ExternalStorageEngine

View File

@ -3,7 +3,6 @@
#include <Storages/IStorage.h>
#include <Common/logger_useful.h>
#include <boost/noncopyable.hpp>
#include <atomic>
#include <shared_mutex>

View File

@ -1,7 +1,6 @@
#pragma once
#include <optional>
#include <boost/noncopyable.hpp>
#include <Storages/IStorage.h>
@ -9,7 +8,7 @@ namespace DB
{
/* Generates random data for given schema.
*/
class StorageGenerateRandom final : public IStorage, boost::noncopyable
class StorageGenerateRandom final : public IStorage
{
public:
StorageGenerateRandom(

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <QueryPipeline/Pipe.h>
#include <Storages/IStorage.h>
@ -9,7 +8,7 @@ namespace DB
/** Internal temporary storage for table function input(...)
*/
class StorageInput final : public IStorage, boost::noncopyable
class StorageInput final : public IStorage
{
public:
StorageInput(const StorageID & table_id, const ColumnsDescription & columns_);

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Common/RWLock.h>
#include <Storages/StorageSet.h>
#include <Storages/TableLockHolder.h>
@ -22,7 +21,7 @@ using HashJoinPtr = std::shared_ptr<HashJoin>;
*
* When using, JOIN must be of the appropriate type (ANY|ALL LEFT|INNER ...).
*/
class StorageJoin final : public StorageSetOrJoinBase, boost::noncopyable
class StorageJoin final : public StorageSetOrJoinBase
{
public:
StorageJoin(

View File

@ -3,7 +3,6 @@
#include <map>
#include <shared_mutex>
#include <boost/noncopyable.hpp>
#include <Disks/IDisk.h>
#include <Storages/IStorage.h>
#include <Common/FileChecker.h>
@ -19,7 +18,7 @@ namespace DB
* Also implements TinyLog - a table engine that is suitable for small chunks of the log.
* It differs from Log in the absence of mark files.
*/
class StorageLog final : public IStorage, boost::noncopyable
class StorageLog final : public IStorage
{
friend class LogSource;
friend class LogSink;

View File

@ -4,7 +4,6 @@
#if USE_MYSQL
#include <boost/noncopyable.hpp>
#include <Storages/StorageProxy.h>
namespace DB
@ -15,7 +14,7 @@ namespace ErrorCodes
extern const int NOT_IMPLEMENTED;
}
class StorageMaterializedMySQL final : public StorageProxy, boost::noncopyable
class StorageMaterializedMySQL final : public StorageProxy
{
public:
StorageMaterializedMySQL(const StoragePtr & nested_storage_, const IDatabase * database_);

View File

@ -2,7 +2,6 @@
#include <Parsers/IAST_fwd.h>
#include <boost/noncopyable.hpp>
#include <Storages/IStorage.h>
#include <Storages/StorageInMemoryMetadata.h>
@ -10,7 +9,7 @@
namespace DB
{
class StorageMaterializedView final : public IStorage, WithMutableContext, boost::noncopyable
class StorageMaterializedView final : public IStorage, WithMutableContext
{
public:
StorageMaterializedView(

View File

@ -4,7 +4,6 @@
#include <optional>
#include <mutex>
#include <boost/noncopyable.hpp>
#include <Core/NamesAndTypes.h>
#include <Storages/IStorage.h>
@ -18,7 +17,7 @@ namespace DB
* It does not support keys.
* Data is stored as a set of blocks and is not stored anywhere else.
*/
class StorageMemory final : public IStorage, boost::noncopyable
class StorageMemory final : public IStorage
{
friend class MemorySink;
friend class MemoryRestoreTask;

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Common/OptimizedRegularExpression.h>
#include <Storages/IStorage.h>
@ -11,7 +10,7 @@ namespace DB
/** A table that represents the union of an arbitrary number of other tables.
* All tables must have the same structure.
*/
class StorageMerge final : public IStorage, WithContext, boost::noncopyable
class StorageMerge final : public IStorage, WithContext
{
public:
using DBToTableSetMap = std::map<String, std::set<String>>;

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Core/Names.h>
#include <Storages/AlterCommands.h>
#include <Storages/IStorage.h>
@ -25,7 +24,7 @@ namespace DB
/** See the description of the data structure in MergeTreeData.
*/
class StorageMergeTree final : public MergeTreeData, boost::noncopyable
class StorageMergeTree final : public MergeTreeData
{
public:
/** Attach the table with the appropriate name, along the appropriate path (with / at the end),

View File

@ -2,7 +2,6 @@
#include <Poco/MongoDB/Connection.h>
#include <boost/noncopyable.hpp>
#include <Storages/IStorage.h>
#include <Storages/ExternalDataSourceConfiguration.h>
@ -13,7 +12,7 @@ namespace DB
* Read only.
*/
class StorageMongoDB final : public IStorage, boost::noncopyable
class StorageMongoDB final : public IStorage
{
public:
StorageMongoDB(

View File

@ -4,7 +4,6 @@
#if USE_MYSQL
#include <boost/noncopyable.hpp>
#include <Storages/IStorage.h>
#include <Storages/MySQL/MySQLSettings.h>
#include <Storages/ExternalDataSourceConfiguration.h>
@ -22,7 +21,7 @@ namespace DB
* Use ENGINE = mysql(host_port, database_name, table_name, user_name, password)
* Read only.
*/
class StorageMySQL final : public IStorage, WithContext, boost::noncopyable
class StorageMySQL final : public IStorage, WithContext
{
public:
StorageMySQL(

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Core/NamesAndTypes.h>
#include <Storages/IStorage.h>
#include <Processors/Sources/NullSource.h>
@ -14,7 +13,7 @@ namespace DB
/** When writing, does nothing.
* When reading, returns nothing.
*/
class StorageNull final : public IStorage, boost::noncopyable
class StorageNull final : public IStorage
{
public:
StorageNull(

View File

@ -3,7 +3,6 @@
#include "config_core.h"
#if USE_LIBPQXX
#include <boost/noncopyable.hpp>
#include <Interpreters/Context.h>
#include <Storages/IStorage.h>
#include <Core/PostgreSQL/PoolWithFailover.h>
@ -17,7 +16,7 @@ class Logger;
namespace DB
{
class StoragePostgreSQL final : public IStorage, boost::noncopyable
class StoragePostgreSQL final : public IStorage
{
public:
StoragePostgreSQL(

View File

@ -3,7 +3,6 @@
#include <base/UUID.h>
#include <atomic>
#include <pcg_random.hpp>
#include <boost/noncopyable.hpp>
#include <Storages/IStorage.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergeTreeDataMergerMutator.h>
@ -81,7 +80,7 @@ namespace DB
* as the time will take the time of creation the appropriate part on any of the replicas.
*/
class StorageReplicatedMergeTree final : public MergeTreeData, boost::noncopyable
class StorageReplicatedMergeTree final : public MergeTreeData
{
public:
enum RenamingRestrictions

View File

@ -11,7 +11,6 @@
#include <Storages/IStorage.h>
#include <Storages/StorageS3Settings.h>
#include <boost/noncopyable.hpp>
#include <Processors/Sources/SourceWithProgress.h>
#include <Poco/URI.h>
#include <Common/logger_useful.h>
@ -120,7 +119,7 @@ private:
* It sends HTTP GET to server when select is called and
* HTTP PUT when insert is called.
*/
class StorageS3 : public IStorage, WithContext, boost::noncopyable
class StorageS3 : public IStorage, WithContext
{
public:
StorageS3(

View File

@ -7,7 +7,6 @@
#include <memory>
#include <optional>
#include <boost/noncopyable.hpp>
#include "Client/Connection.h"
#include <Interpreters/Cluster.h>
#include <IO/S3Common.h>
@ -18,7 +17,7 @@ namespace DB
class Context;
class StorageS3Cluster : public IStorage, boost::noncopyable
class StorageS3Cluster : public IStorage
{
public:
StorageS3Cluster(

View File

@ -3,7 +3,6 @@
#include "config_core.h"
#if USE_SQLITE
#include <boost/noncopyable.hpp>
#include <Storages/IStorage.h>
#include <sqlite3.h>
@ -16,7 +15,7 @@ class Logger;
namespace DB
{
class StorageSQLite final : public IStorage, public WithContext, boost::noncopyable
class StorageSQLite final : public IStorage, public WithContext
{
public:
using SQLitePtr = std::shared_ptr<sqlite3>;

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Interpreters/Context.h>
#include <Storages/IStorage.h>
#include <Storages/SetSettings.h>
@ -62,7 +61,7 @@ private:
* and also written to a file-backup, for recovery after a restart.
* Reading from the table is not possible directly - it is possible to specify only the right part of the IN statement.
*/
class StorageSet final : public StorageSetOrJoinBase, boost::noncopyable
class StorageSet final : public StorageSetOrJoinBase
{
public:
StorageSet(

View File

@ -3,7 +3,6 @@
#include <map>
#include <shared_mutex>
#include <boost/noncopyable.hpp>
#include <Core/Defines.h>
#include <Storages/IStorage.h>
#include <Formats/IndexForNativeFormat.h>
@ -19,7 +18,7 @@ struct IndexForNativeFormat;
/** Implements a table engine that is suitable for small chunks of the log.
* In doing so, stores all the columns in a single Native file, with a nearby index.
*/
class StorageStripeLog final : public IStorage, boost::noncopyable
class StorageStripeLog final : public IStorage
{
friend class StripeLogSource;
friend class StripeLogSink;

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/IStorage.h>
#include <Poco/URI.h>
#include <Processors/Sinks/SinkToStorage.h>
@ -122,7 +121,7 @@ private:
OutputFormatPtr writer;
};
class StorageURL : public IStorageURLBase, boost::noncopyable
class StorageURL : public IStorageURLBase
{
public:
StorageURL(

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/IStorage.h>
@ -9,7 +8,7 @@ namespace DB
/* One block storage used for values table function
* It's structure is similar to IStorageSystemOneBlock
*/
class StorageValues final : public IStorage, boost::noncopyable
class StorageValues final : public IStorage
{
public:
StorageValues(

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/IAST_fwd.h>
#include <Storages/IStorage.h>
@ -9,7 +8,7 @@
namespace DB
{
class StorageView final : public IStorage, boost::noncopyable
class StorageView final : public IStorage
{
public:
StorageView(

View File

@ -1,13 +1,12 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Storages/System/IStorageSystemOneBlock.h>
namespace DB
{
class StorageSystemAggregateFunctionCombinators final : public IStorageSystemOneBlock<StorageSystemAggregateFunctionCombinators>, boost::noncopyable
class StorageSystemAggregateFunctionCombinators final : public IStorageSystemOneBlock<StorageSystemAggregateFunctionCombinators>
{
protected:
void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override;

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
namespace DB
@ -9,7 +8,7 @@ namespace DB
/** Implements the system table `asynhronous_inserts`,
* which contains information about pending asynchronous inserts in queue.
*/
class StorageSystemAsynchronousInserts final : public IStorageSystemOneBlock<StorageSystemAsynchronousInserts>, boost::noncopyable
class StorageSystemAsynchronousInserts final : public IStorageSystemOneBlock<StorageSystemAsynchronousInserts>
{
public:
std::string getName() const override { return "SystemAsynchronousInserts"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
namespace DB
@ -12,7 +11,7 @@ class Context;
/** Implements system table asynchronous_metrics, which allows to get values of periodically (asynchronously) updated metrics.
*/
class StorageSystemAsynchronousMetrics final : public IStorageSystemOneBlock<StorageSystemAsynchronousMetrics>, boost::noncopyable
class StorageSystemAsynchronousMetrics final : public IStorageSystemOneBlock<StorageSystemAsynchronousMetrics>
{
public:
StorageSystemAsynchronousMetrics(const StorageID & table_id_, const AsynchronousMetrics & async_metrics_);

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -8,7 +7,7 @@ namespace DB
{
/// Implements `grants` system table, which allows you to get information about grants.
class StorageSystemBackups final : public boost::noncopyable, public IStorageSystemOneBlock<StorageSystemBackups>
class StorageSystemBackups final : public IStorageSystemOneBlock<StorageSystemBackups>
{
public:
std::string getName() const override { return "SystemBackups"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -12,7 +11,7 @@ class Context;
/** System table "build_options" with many params used for clickhouse building
*/
class StorageSystemBuildOptions final : public IStorageSystemOneBlock<StorageSystemBuildOptions>, boost::noncopyable
class StorageSystemBuildOptions final : public IStorageSystemOneBlock<StorageSystemBuildOptions>
{
protected:
void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override;

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -16,7 +15,7 @@ class Cluster;
* that allows to obtain information about available clusters
* (which may be specified in Distributed tables).
*/
class StorageSystemClusters final : public IStorageSystemOneBlock<StorageSystemClusters>, boost::noncopyable
class StorageSystemClusters final : public IStorageSystemOneBlock<StorageSystemClusters>
{
public:
std::string getName() const override { return "SystemClusters"; }

View File

@ -1,12 +1,11 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
namespace DB
{
class StorageSystemCollations final : public IStorageSystemOneBlock<StorageSystemCollations>, boost::noncopyable
class StorageSystemCollations final : public IStorageSystemOneBlock<StorageSystemCollations>
{
protected:
void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override;

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -11,7 +10,7 @@ class Context;
/** Implements system table 'columns', that allows to get information about columns for every table.
*/
class StorageSystemColumns final : public IStorage, boost::noncopyable
class StorageSystemColumns final : public IStorage
{
public:
explicit StorageSystemColumns(const StorageID & table_id_);

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
namespace DB
@ -10,7 +9,7 @@ class Context;
/** System table "contributors" with list of clickhouse contributors
*/
class StorageSystemContributors final : public IStorageSystemOneBlock<StorageSystemContributors>, boost::noncopyable
class StorageSystemContributors final : public IStorageSystemOneBlock<StorageSystemContributors>
{
protected:
void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override;

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -9,7 +8,7 @@ namespace DB
class Context;
/// Implements `current_roles` system table, which allows you to get information about current roles.
class StorageSystemCurrentRoles final : public IStorageSystemOneBlock<StorageSystemCurrentRoles>, boost::noncopyable
class StorageSystemCurrentRoles final : public IStorageSystemOneBlock<StorageSystemCurrentRoles>
{
public:
std::string getName() const override { return "SystemCurrentRoles"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Interpreters/DDLWorker.h>
#include <Storages/System/IStorageSystemOneBlock.h>
#include <future>
@ -12,7 +11,7 @@ class Context;
/** System table "distributed_ddl_queue" with list of queries that are currently in the DDL worker queue.
*/
class StorageSystemDDLWorkerQueue final : public IStorageSystemOneBlock<StorageSystemDDLWorkerQueue>, boost::noncopyable
class StorageSystemDDLWorkerQueue final : public IStorageSystemOneBlock<StorageSystemDDLWorkerQueue>
{
protected:
void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override;

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/IStorage.h>
@ -8,7 +7,7 @@ namespace DB
{
/// For system.data_skipping_indices table - describes the data skipping indices in tables, similar to system.columns.
class StorageSystemDataSkippingIndices : public IStorage, boost::noncopyable
class StorageSystemDataSkippingIndices : public IStorage
{
public:
explicit StorageSystemDataSkippingIndices(const StorageID & table_id_);

View File

@ -1,12 +1,11 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
namespace DB
{
class StorageSystemDataTypeFamilies final : public IStorageSystemOneBlock<StorageSystemDataTypeFamilies>, boost::noncopyable
class StorageSystemDataTypeFamilies final : public IStorageSystemOneBlock<StorageSystemDataTypeFamilies>
{
protected:
void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override;

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -12,7 +11,7 @@ class Context;
/** Implements `databases` system table, which allows you to get information about all databases.
*/
class StorageSystemDatabases final : public IStorageSystemOneBlock<StorageSystemDatabases>, boost::noncopyable
class StorageSystemDatabases final : public IStorageSystemOneBlock<StorageSystemDatabases>
{
public:
std::string getName() const override

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/IStorage.h>
namespace DB
@ -12,7 +11,7 @@ namespace DB
* We don't use StorageSystemPartsBase, because it introduces virtual _state
* column and column aliases which we don't need.
*/
class StorageSystemDetachedParts final : public IStorage, boost::noncopyable
class StorageSystemDetachedParts final : public IStorage
{
public:
explicit StorageSystemDetachedParts(const StorageID & table_id_);

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -10,7 +9,7 @@ namespace DB
class Context;
class StorageSystemDictionaries final : public IStorageSystemOneBlock<StorageSystemDictionaries>, boost::noncopyable
class StorageSystemDictionaries final : public IStorageSystemOneBlock<StorageSystemDictionaries>
{
public:
std::string getName() const override { return "SystemDictionaries"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Formats/FormatSettings.h>
#include <Storages/IStorage.h>
#include <Storages/MergeTree/MergeTreeData.h>
@ -14,7 +13,7 @@ class Context;
/** Implements the system table `disks`, which allows you to get information about all disks.
*/
class StorageSystemDisks final : public IStorage, boost::noncopyable
class StorageSystemDisks final : public IStorage
{
public:
explicit StorageSystemDisks(const StorageID & table_id_);

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -12,7 +11,7 @@ class Context;
/** Implements the `distribution_queue` system table, which allows you to view the INSERT queues for the Distributed tables.
*/
class StorageSystemDistributionQueue final : public IStorageSystemOneBlock<StorageSystemDistributionQueue>, boost::noncopyable
class StorageSystemDistributionQueue final : public IStorageSystemOneBlock<StorageSystemDistributionQueue>
{
public:
std::string getName() const override { return "SystemDistributionQueue"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -9,7 +8,7 @@ namespace DB
class Context;
/// Implements `enabled_roles` system table, which allows you to get information about enabled roles.
class StorageSystemEnabledRoles final : public IStorageSystemOneBlock<StorageSystemEnabledRoles>, boost::noncopyable
class StorageSystemEnabledRoles final : public IStorageSystemOneBlock<StorageSystemEnabledRoles>
{
public:
std::string getName() const override { return "SystemEnabledRoles"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -14,7 +13,7 @@ class Context;
* Implements the `errors` system table, which shows the error code and the number of times it happens
* (i.e. Exception with this code had been thrown).
*/
class StorageSystemErrors final : public IStorageSystemOneBlock<StorageSystemErrors>, boost::noncopyable
class StorageSystemErrors final : public IStorageSystemOneBlock<StorageSystemErrors>
{
public:
std::string getName() const override { return "SystemErrors"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
namespace DB
@ -11,7 +10,7 @@ class Context;
/** Implements `events` system table, which allows you to obtain information for profiling.
*/
class StorageSystemEvents final : public IStorageSystemOneBlock<StorageSystemEvents>, boost::noncopyable
class StorageSystemEvents final : public IStorageSystemOneBlock<StorageSystemEvents>
{
public:
std::string getName() const override { return "SystemEvents"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
namespace DB
@ -29,7 +28,7 @@ namespace DB
* FORMAT Vertical
*/
class StorageSystemFilesystemCache final : public IStorageSystemOneBlock<StorageSystemFilesystemCache>, boost::noncopyable
class StorageSystemFilesystemCache final : public IStorageSystemOneBlock<StorageSystemFilesystemCache>
{
public:
explicit StorageSystemFilesystemCache(const StorageID & table_id_);

View File

@ -1,11 +1,10 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
namespace DB
{
class StorageSystemFormats final : public IStorageSystemOneBlock<StorageSystemFormats>, boost::noncopyable
class StorageSystemFormats final : public IStorageSystemOneBlock<StorageSystemFormats>
{
protected:
void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override;

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -13,7 +12,7 @@ class Context;
/** Implements `functions`system table, which allows you to get a list
* all normal and aggregate functions.
*/
class StorageSystemFunctions final : public IStorageSystemOneBlock<StorageSystemFunctions>, boost::noncopyable
class StorageSystemFunctions final : public IStorageSystemOneBlock<StorageSystemFunctions>
{
public:
std::string getName() const override { return "SystemFunctions"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -9,7 +8,7 @@ namespace DB
class Context;
/// Implements `grants` system table, which allows you to get information about grants.
class StorageSystemGrants final : public IStorageSystemOneBlock<StorageSystemGrants>, boost::noncopyable
class StorageSystemGrants final : public IStorageSystemOneBlock<StorageSystemGrants>
{
public:
std::string getName() const override { return "SystemGrants"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
@ -11,7 +10,7 @@ namespace DB
{
/// Provides information about Graphite configuration.
class StorageSystemGraphite final : public IStorageSystemOneBlock<StorageSystemGraphite>, boost::noncopyable
class StorageSystemGraphite final : public IStorageSystemOneBlock<StorageSystemGraphite>
{
public:
std::string getName() const override { return "SystemGraphite"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -11,7 +10,7 @@ class Context;
/** System table "licenses" with list of licenses of 3rd party libraries
*/
class StorageSystemLicenses final : public IStorageSystemOneBlock<StorageSystemLicenses>, boost::noncopyable
class StorageSystemLicenses final : public IStorageSystemOneBlock<StorageSystemLicenses>
{
protected:
void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override;

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <DataTypes/DataTypeString.h>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -13,7 +12,7 @@ class Context;
/** Information about macros for introspection.
*/
class StorageSystemMacros final : public IStorageSystemOneBlock<StorageSystemMacros>, boost::noncopyable
class StorageSystemMacros final : public IStorageSystemOneBlock<StorageSystemMacros>
{
public:
std::string getName() const override { return "SystemMacros"; }

View File

@ -3,7 +3,6 @@
#include "config_core.h"
#if USE_ROCKSDB
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -13,7 +12,7 @@ class Context;
/// Implements `merge_tree_metadata_cache` system table, which allows you to view the metadata cache data in rocksdb for testing purposes.
class StorageSystemMergeTreeMetadataCache : public IStorageSystemOneBlock<StorageSystemMergeTreeMetadataCache>, boost::noncopyable
class StorageSystemMergeTreeMetadataCache : public IStorageSystemOneBlock<StorageSystemMergeTreeMetadataCache>
{
public:
std::string getName() const override { return "SystemMergeTreeMetadataCache"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/MergeTree/MergeTreeSettings.h>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -15,7 +14,7 @@ class Context;
* which allows to get information about the current MergeTree settings.
*/
template <bool replicated>
class SystemMergeTreeSettings final : public IStorageSystemOneBlock<SystemMergeTreeSettings<replicated>>, boost::noncopyable
class SystemMergeTreeSettings final : public IStorageSystemOneBlock<SystemMergeTreeSettings<replicated>>
{
public:
std::string getName() const override { return replicated ? "SystemReplicatedMergeTreeSettings" : "SystemMergeTreeSettings"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeArray.h>
@ -13,7 +12,7 @@ namespace DB
class Context;
class StorageSystemMerges final : public IStorageSystemOneBlock<StorageSystemMerges>, boost::noncopyable
class StorageSystemMerges final : public IStorageSystemOneBlock<StorageSystemMerges>
{
public:
std::string getName() const override { return "SystemMerges"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -12,7 +11,7 @@ class Context;
/** Implements `metrics` system table, which provides information about the operation of the server.
*/
class StorageSystemMetrics final : public IStorageSystemOneBlock<StorageSystemMetrics>, boost::noncopyable
class StorageSystemMetrics final : public IStorageSystemOneBlock<StorageSystemMetrics>
{
public:
std::string getName() const override { return "SystemMetrics"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -10,7 +9,7 @@ namespace DB
class Context;
class StorageSystemModels final : public IStorageSystemOneBlock<StorageSystemModels>, boost::noncopyable
class StorageSystemModels final : public IStorageSystemOneBlock<StorageSystemModels>
{
public:
std::string getName() const override { return "SystemModels"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -12,7 +11,7 @@ class Context;
/// Implements the `mutations` system table, which provides information about the status of mutations
/// in the MergeTree tables.
class StorageSystemMutations final : public IStorageSystemOneBlock<StorageSystemMutations>, boost::noncopyable
class StorageSystemMutations final : public IStorageSystemOneBlock<StorageSystemMutations>
{
public:
String getName() const override { return "SystemMutations"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <optional>
#include <Storages/IStorage.h>
@ -23,7 +22,7 @@ class Context;
* In multithreaded case, if even_distributed is False, implementation with atomic is used,
* and result is always in [0 ... limit - 1] range.
*/
class StorageSystemNumbers final : public IStorage, boost::noncopyable
class StorageSystemNumbers final : public IStorage
{
public:
/// If even_distribution is true, numbers are distributed evenly between streams.

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/IStorage.h>
@ -15,7 +14,7 @@ class Context;
* Used when the table is not specified in the query.
* Analog of the DUAL table in Oracle and MySQL.
*/
class StorageSystemOne final : public IStorage, boost::noncopyable
class StorageSystemOne final : public IStorage
{
public:
explicit StorageSystemOne(const StorageID & table_id_);

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -10,7 +9,7 @@ namespace DB
class Context;
class StorageSystemPartMovesBetweenShards final : public IStorageSystemOneBlock<StorageSystemPartMovesBetweenShards>, boost::noncopyable
class StorageSystemPartMovesBetweenShards final : public IStorageSystemOneBlock<StorageSystemPartMovesBetweenShards>
{
public:
std::string getName() const override { return "SystemShardMoves"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/StorageSystemPartsBase.h>
@ -12,7 +11,7 @@ class Context;
/** Implements system table 'parts' which allows to get information about data parts for tables of MergeTree family.
*/
class StorageSystemParts final : public StorageSystemPartsBase, boost::noncopyable
class StorageSystemParts final : public StorageSystemPartsBase
{
public:
explicit StorageSystemParts(const StorageID & table_id_);

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/StorageSystemPartsBase.h>
@ -13,7 +12,7 @@ class Context;
/** Implements system table 'parts_columns' which allows to get information about
* columns in data parts for tables of MergeTree family.
*/
class StorageSystemPartsColumns final : public StorageSystemPartsBase, boost::noncopyable
class StorageSystemPartsColumns final : public StorageSystemPartsBase
{
public:
explicit StorageSystemPartsColumns(const StorageID & table_id_);

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -9,7 +8,7 @@ namespace DB
class Context;
/// Implements `privileges` system table, which allows you to get information about access types.
class StorageSystemPrivileges final : public IStorageSystemOneBlock<StorageSystemPrivileges>, boost::noncopyable
class StorageSystemPrivileges final : public IStorageSystemOneBlock<StorageSystemPrivileges>
{
public:
std::string getName() const override { return "SystemPrivileges"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -12,7 +11,7 @@ class Context;
/** Implements `processes` system table, which allows you to get information about the queries that are currently executing.
*/
class StorageSystemProcesses final : public IStorageSystemOneBlock<StorageSystemProcesses>, boost::noncopyable
class StorageSystemProcesses final : public IStorageSystemOneBlock<StorageSystemProcesses>
{
public:
std::string getName() const override { return "SystemProcesses"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/StorageSystemPartsBase.h>
@ -12,7 +11,7 @@ class Context;
/** Implements system table 'projection_parts' which allows to get information about projection parts for tables of MergeTree family.
*/
class StorageSystemProjectionParts final : public StorageSystemPartsBase, boost::noncopyable
class StorageSystemProjectionParts final : public StorageSystemPartsBase
{
public:
explicit StorageSystemProjectionParts(const StorageID & table_id_);

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/StorageSystemPartsBase.h>
@ -13,7 +12,7 @@ class Context;
/** Implements system table 'projection_parts_columns' which allows to get information about
* columns in projection parts for tables of MergeTree family.
*/
class StorageSystemProjectionPartsColumns final : public StorageSystemPartsBase, boost::noncopyable
class StorageSystemProjectionPartsColumns final : public StorageSystemPartsBase
{
public:
explicit StorageSystemProjectionPartsColumns(const StorageID & table_id_);

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -9,7 +8,7 @@ namespace DB
class Context;
/// Implements `quota_limits` system table, which allows you to get information about the limits set for quotas.
class StorageSystemQuotaLimits final : public IStorageSystemOneBlock<StorageSystemQuotaLimits>, boost::noncopyable
class StorageSystemQuotaLimits final : public IStorageSystemOneBlock<StorageSystemQuotaLimits>
{
public:
std::string getName() const override { return "SystemQuotaLimits"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -13,7 +12,7 @@ struct QuotaUsage;
/** Implements the `quota_usage` system table, which allows you to get information about
* how the current user uses the quota.
*/
class StorageSystemQuotaUsage final : public IStorageSystemOneBlock<StorageSystemQuotaUsage>, boost::noncopyable
class StorageSystemQuotaUsage final : public IStorageSystemOneBlock<StorageSystemQuotaUsage>
{
public:
std::string getName() const override { return "SystemQuotaUsage"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -10,7 +9,7 @@ class Context;
/** Implements the `quotas` system tables, which allows you to get information about quotas.
*/
class StorageSystemQuotas final : public IStorageSystemOneBlock<StorageSystemQuotas>, boost::noncopyable
class StorageSystemQuotas final : public IStorageSystemOneBlock<StorageSystemQuotas>
{
public:
std::string getName() const override { return "SystemQuotas"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -11,7 +10,7 @@ class Context;
/** Implements the `quotas_usage` system table, which allows you to get information about
* how all users use the quotas.
*/
class StorageSystemQuotasUsage final : public IStorageSystemOneBlock<StorageSystemQuotasUsage>, boost::noncopyable
class StorageSystemQuotasUsage final : public IStorageSystemOneBlock<StorageSystemQuotasUsage>
{
public:
std::string getName() const override { return "SystemQuotasUsage"; }

View File

@ -1,12 +1,11 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
namespace DB
{
class StorageSystemRemoteDataPaths : public IStorage, boost::noncopyable
class StorageSystemRemoteDataPaths : public IStorage
{
public:
explicit StorageSystemRemoteDataPaths(const StorageID & table_id_);

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/IStorage.h>
@ -12,7 +11,7 @@ class Context;
/** Implements `replicas` system table, which provides information about the status of the replicated tables.
*/
class StorageSystemReplicas final : public IStorage, boost::noncopyable
class StorageSystemReplicas final : public IStorage
{
public:
explicit StorageSystemReplicas(const StorageID & table_id_);

View File

@ -1,7 +1,6 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -11,7 +10,7 @@ namespace DB
class Context;
/// system.replicated_fetches table. Takes data from context.getReplicatedFetchList()
class StorageSystemReplicatedFetches final : public IStorageSystemOneBlock<StorageSystemReplicatedFetches>, boost::noncopyable
class StorageSystemReplicatedFetches final : public IStorageSystemOneBlock<StorageSystemReplicatedFetches>
{
public:
std::string getName() const override { return "SystemReplicatedFetches"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -12,7 +11,7 @@ class Context;
/** Implements the `replication_queue` system table, which allows you to view the replication queues for the replicated tables.
*/
class StorageSystemReplicationQueue final : public IStorageSystemOneBlock<StorageSystemReplicationQueue>, boost::noncopyable
class StorageSystemReplicationQueue final : public IStorageSystemOneBlock<StorageSystemReplicationQueue>
{
public:
std::string getName() const override { return "SystemReplicationQueue"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -9,7 +8,7 @@ namespace DB
class Context;
/// Implements `role_grants` system table, which allows you to get information about granted roles.
class StorageSystemRoleGrants final : public IStorageSystemOneBlock<StorageSystemRoleGrants>, boost::noncopyable
class StorageSystemRoleGrants final : public IStorageSystemOneBlock<StorageSystemRoleGrants>
{
public:
std::string getName() const override { return "SystemRoleGrants"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -9,7 +8,7 @@ namespace DB
class Context;
/// Implements `roles` system table, which allows you to get information about roles.
class StorageSystemRoles final : public IStorageSystemOneBlock<StorageSystemRoles>, boost::noncopyable
class StorageSystemRoles final : public IStorageSystemOneBlock<StorageSystemRoles>
{
public:
std::string getName() const override { return "SystemRoles"; }

View File

@ -1,6 +1,5 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Storages/System/IStorageSystemOneBlock.h>
@ -11,7 +10,7 @@ class Context;
/// Implements `row_policies` system table, which allows you to get information about row policies.
class StorageSystemRowPolicies final : public IStorageSystemOneBlock<StorageSystemRowPolicies>, boost::noncopyable
class StorageSystemRowPolicies final : public IStorageSystemOneBlock<StorageSystemRowPolicies>
{
public:
std::string getName() const override { return "SystemRowPolicies"; }

Some files were not shown because too many files have changed in this diff Show More