mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Using std::make_unique [#METR-2807].
This commit is contained in:
parent
2d37140ee1
commit
a9eb28c07d
@ -258,9 +258,9 @@ protected:
|
||||
{
|
||||
Settings * parallel_replicas_settings = send_settings ? &settings : nullptr;
|
||||
if (connection != nullptr)
|
||||
parallel_replicas = ext::make_unique<ParallelReplicas>(connection, parallel_replicas_settings, throttler);
|
||||
parallel_replicas = std::make_unique<ParallelReplicas>(connection, parallel_replicas_settings, throttler);
|
||||
else
|
||||
parallel_replicas = ext::make_unique<ParallelReplicas>(pool, parallel_replicas_settings, throttler);
|
||||
parallel_replicas = std::make_unique<ParallelReplicas>(pool, parallel_replicas_settings, throttler);
|
||||
}
|
||||
|
||||
/// Возвращает true, если запрос отправлен, а ещё не выполнен.
|
||||
|
@ -7,7 +7,7 @@
|
||||
#include <DB/Dictionaries/ClickHouseDictionarySource.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <Yandex/singleton.h>
|
||||
#include <statdaemons/ext/memory.hpp>
|
||||
#include <memory>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -63,15 +63,15 @@ public:
|
||||
{
|
||||
const auto filename = config.getString(config_prefix + ".file.path");
|
||||
const auto format = config.getString(config_prefix + ".file.format");
|
||||
return ext::make_unique<FileDictionarySource>(filename, format, sample_block, context);
|
||||
return std::make_unique<FileDictionarySource>(filename, format, sample_block, context);
|
||||
}
|
||||
else if ("mysql" == source_type)
|
||||
{
|
||||
return ext::make_unique<MySQLDictionarySource>(config, config_prefix + ".mysql", sample_block);
|
||||
return std::make_unique<MySQLDictionarySource>(config, config_prefix + ".mysql", sample_block);
|
||||
}
|
||||
else if ("clickhouse" == source_type)
|
||||
{
|
||||
return ext::make_unique<ClickHouseDictionarySource>(config, config_prefix + ".clickhouse",
|
||||
return std::make_unique<ClickHouseDictionarySource>(config, config_prefix + ".clickhouse",
|
||||
sample_block, context);
|
||||
}
|
||||
|
||||
|
@ -30,7 +30,7 @@ public:
|
||||
|
||||
BlockInputStreamPtr loadAll() override
|
||||
{
|
||||
auto in_ptr = ext::make_unique<ReadBufferFromFile>(filename);
|
||||
auto in_ptr = std::make_unique<ReadBufferFromFile>(filename);
|
||||
auto stream = context.getFormatFactory().getInput(
|
||||
format, *in_ptr, sample_block, max_block_size, context.getDataTypeFactory());
|
||||
last_modification = getLastModification();
|
||||
@ -57,7 +57,7 @@ public:
|
||||
bool isModified() const override { return getLastModification() > last_modification; }
|
||||
bool supportsSelectiveLoad() const override { return false; }
|
||||
|
||||
DictionarySourcePtr clone() const override { return ext::make_unique<FileDictionarySource>(*this); }
|
||||
DictionarySourcePtr clone() const override { return std::make_unique<FileDictionarySource>(*this); }
|
||||
|
||||
private:
|
||||
Poco::Timestamp getLastModification() const { return Poco::File{filename}.getLastModified(); }
|
||||
|
@ -34,7 +34,7 @@ public:
|
||||
|
||||
bool isCached() const override { return false; }
|
||||
|
||||
DictionaryPtr clone() const override { return ext::make_unique<FlatDictionary>(*this); }
|
||||
DictionaryPtr clone() const override { return std::make_unique<FlatDictionary>(*this); }
|
||||
|
||||
const IDictionarySource * getSource() const override { return source_ptr.get(); }
|
||||
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <DB/Dictionaries/DictionaryStructure.h>
|
||||
#include <DB/Common/HashTable/HashMap.h>
|
||||
#include <statdaemons/ext/range.hpp>
|
||||
#include <statdaemons/ext/memory.hpp>
|
||||
#include <memory>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -32,7 +32,7 @@ public:
|
||||
|
||||
bool isCached() const override { return false; }
|
||||
|
||||
DictionaryPtr clone() const override { return ext::make_unique<HashedDictionary>(*this); }
|
||||
DictionaryPtr clone() const override { return std::make_unique<HashedDictionary>(*this); }
|
||||
|
||||
const IDictionarySource * getSource() const override { return source_ptr.get(); }
|
||||
|
||||
|
@ -3,7 +3,7 @@
|
||||
#include <Poco/Mutex.h>
|
||||
|
||||
#include <statdaemons/OptimizedRegularExpression.h>
|
||||
#include <statdaemons/ext/memory.hpp>
|
||||
#include <memory>
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
@ -313,7 +313,7 @@ namespace Regexps
|
||||
|
||||
auto it = known_regexps.find(pattern);
|
||||
if (known_regexps.end() == it)
|
||||
it = known_regexps.emplace(pattern, ext::make_unique<Holder>()).first;
|
||||
it = known_regexps.emplace(pattern, std::make_unique<Holder>()).first;
|
||||
|
||||
return it->second->get([&pattern] {
|
||||
return new Regexp{createRegexp<like>(pattern)};
|
||||
|
@ -16,7 +16,7 @@ inline void evaluateMissingDefaults(Block & block,
|
||||
if (column_defaults.empty())
|
||||
return;
|
||||
|
||||
ASTPtr default_expr_list{ext::make_unique<ASTExpressionList>().release()};
|
||||
ASTPtr default_expr_list{std::make_unique<ASTExpressionList>().release()};
|
||||
|
||||
for (const auto & column : required_columns)
|
||||
{
|
||||
|
@ -10,7 +10,7 @@
|
||||
#include <DB/Interpreters/InterpreterInsertQuery.h>
|
||||
|
||||
#include <statdaemons/Increment.h>
|
||||
#include <statdaemons/ext/memory.hpp>
|
||||
#include <memory>
|
||||
#include <Yandex/Revision.h>
|
||||
|
||||
#include <iostream>
|
||||
|
@ -14,7 +14,7 @@
|
||||
#include <DB/Storages/AlterCommands.h>
|
||||
#include <Poco/File.h>
|
||||
#include <Poco/RWLock.h>
|
||||
#include <statdaemons/ext/memory.hpp>
|
||||
#include <memory>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -129,7 +129,7 @@ public:
|
||||
*/
|
||||
TableDataWriteLockPtr lockDataForAlter()
|
||||
{
|
||||
auto res = ext::make_unique<Poco::ScopedWriteRWLock>(data_lock);
|
||||
auto res = std::make_unique<Poco::ScopedWriteRWLock>(data_lock);
|
||||
if (is_dropped)
|
||||
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);
|
||||
return res;
|
||||
@ -137,7 +137,7 @@ public:
|
||||
|
||||
TableStructureWriteLockPtr lockStructureForAlter()
|
||||
{
|
||||
auto res = ext::make_unique<Poco::ScopedWriteRWLock>(structure_lock);
|
||||
auto res = std::make_unique<Poco::ScopedWriteRWLock>(structure_lock);
|
||||
if (is_dropped)
|
||||
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);
|
||||
return res;
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <statdaemons/Stopwatch.h>
|
||||
#include <statdaemons/ext/memory.hpp>
|
||||
#include <memory>
|
||||
#include <list>
|
||||
#include <mutex>
|
||||
#include <atomic>
|
||||
@ -67,7 +67,7 @@ public:
|
||||
EntryPtr insert(Args &&... args)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock{mutex};
|
||||
return ext::make_unique<Entry>(*this, merges.emplace(merges.end(), std::forward<Args>(args)...));
|
||||
return std::make_unique<Entry>(*this, merges.emplace(merges.end(), std::forward<Args>(args)...));
|
||||
}
|
||||
|
||||
container_t get() const
|
||||
|
@ -9,7 +9,7 @@
|
||||
#include <DB/Parsers/ASTSubquery.h>
|
||||
#include <DB/Parsers/formatAST.h>
|
||||
#include <DB/Common/escapeForFileName.h>
|
||||
#include <statdaemons/ext/memory.hpp>
|
||||
#include <memory>
|
||||
#include <unordered_map>
|
||||
#include <map>
|
||||
#include <limits>
|
||||
|
@ -11,7 +11,7 @@
|
||||
#include <DB/DataTypes/DataTypeNested.h>
|
||||
|
||||
#include <DB/Parsers/ASTExpressionList.h>
|
||||
#include <statdaemons/ext/memory.hpp>
|
||||
#include <memory>
|
||||
|
||||
#include <DB/Parsers/formatAST.h>
|
||||
|
||||
|
@ -4,7 +4,7 @@
|
||||
#include <DB/Dictionaries/HashedDictionary.h>
|
||||
#include <DB/Dictionaries/CacheDictionary.h>
|
||||
#include <DB/Dictionaries/DictionaryStructure.h>
|
||||
#include <statdaemons/ext/memory.hpp>
|
||||
#include <memory>
|
||||
#include <Yandex/singleton.h>
|
||||
|
||||
namespace DB
|
||||
@ -33,11 +33,11 @@ DictionaryPtr DictionaryFactory::create(const std::string & name, Poco::Util::Ab
|
||||
|
||||
if ("flat" == layout_type)
|
||||
{
|
||||
return ext::make_unique<FlatDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime);
|
||||
return std::make_unique<FlatDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime);
|
||||
}
|
||||
else if ("hashed" == layout_type)
|
||||
{
|
||||
return ext::make_unique<HashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime);
|
||||
return std::make_unique<HashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime);
|
||||
}
|
||||
else if ("cache" == layout_type)
|
||||
{
|
||||
@ -48,7 +48,7 @@ DictionaryPtr DictionaryFactory::create(const std::string & name, Poco::Util::Ab
|
||||
ErrorCodes::TOO_SMALL_BUFFER_SIZE
|
||||
};
|
||||
|
||||
return ext::make_unique<CacheDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, size);
|
||||
return std::make_unique<CacheDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, size);
|
||||
}
|
||||
|
||||
throw Exception{
|
||||
|
@ -8,7 +8,7 @@
|
||||
#include <Yandex/ErrorHandlers.h>
|
||||
#include <Yandex/Revision.h>
|
||||
#include <statdaemons/ConfigProcessor.h>
|
||||
#include <statdaemons/ext/memory.hpp>
|
||||
#include <memory>
|
||||
|
||||
#include <DB/Interpreters/loadMetadata.h>
|
||||
#include <DB/Storages/StorageSystemNumbers.h>
|
||||
@ -491,7 +491,7 @@ int Server::main(const std::vector<std::string> & args)
|
||||
global_context->setMacros(Macros(config(), "macros"));
|
||||
|
||||
std::string users_config_path = config().getString("users_config", config().getString("config-file", "config.xml"));
|
||||
auto users_config_reloader = ext::make_unique<UsersConfigReloader>(users_config_path, global_context.get());
|
||||
auto users_config_reloader = std::make_unique<UsersConfigReloader>(users_config_path, global_context.get());
|
||||
|
||||
/// Максимальное количество одновременно выполняющихся запросов.
|
||||
global_context->getProcessList().setMaxSize(config().getInt("max_concurrent_queries", 0));
|
||||
@ -536,7 +536,7 @@ int Server::main(const std::vector<std::string> & args)
|
||||
|
||||
{
|
||||
const auto profile_events_transmitter = config().getBool("use_graphite", true)
|
||||
? ext::make_unique<ProfileEventsTransmitter>()
|
||||
? std::make_unique<ProfileEventsTransmitter>()
|
||||
: nullptr;
|
||||
|
||||
const std::string listen_host = config().getString("listen_host", "::");
|
||||
|
@ -324,7 +324,7 @@ MergeTreeData::DataPartPtr MergeTreeDataMerger::mergeParts(
|
||||
{
|
||||
MarkRanges ranges(1, MarkRange(0, parts[i]->size));
|
||||
|
||||
auto input = ext::make_unique<MergeTreeBlockInputStream>(
|
||||
auto input = std::make_unique<MergeTreeBlockInputStream>(
|
||||
data.getFullPath() + parts[i]->name + '/', DEFAULT_MERGE_BLOCK_SIZE, union_column_names, data,
|
||||
parts[i], ranges, false, nullptr, "");
|
||||
|
||||
@ -348,19 +348,19 @@ MergeTreeData::DataPartPtr MergeTreeDataMerger::mergeParts(
|
||||
switch (data.mode)
|
||||
{
|
||||
case MergeTreeData::Ordinary:
|
||||
merged_stream = ext::make_unique<MergingSortedBlockInputStream>(src_streams, data.getSortDescription(), DEFAULT_MERGE_BLOCK_SIZE);
|
||||
merged_stream = std::make_unique<MergingSortedBlockInputStream>(src_streams, data.getSortDescription(), DEFAULT_MERGE_BLOCK_SIZE);
|
||||
break;
|
||||
|
||||
case MergeTreeData::Collapsing:
|
||||
merged_stream = ext::make_unique<CollapsingSortedBlockInputStream>(src_streams, data.getSortDescription(), data.sign_column, DEFAULT_MERGE_BLOCK_SIZE);
|
||||
merged_stream = std::make_unique<CollapsingSortedBlockInputStream>(src_streams, data.getSortDescription(), data.sign_column, DEFAULT_MERGE_BLOCK_SIZE);
|
||||
break;
|
||||
|
||||
case MergeTreeData::Summing:
|
||||
merged_stream = ext::make_unique<SummingSortedBlockInputStream>(src_streams, data.getSortDescription(), data.columns_to_sum, DEFAULT_MERGE_BLOCK_SIZE);
|
||||
merged_stream = std::make_unique<SummingSortedBlockInputStream>(src_streams, data.getSortDescription(), data.columns_to_sum, DEFAULT_MERGE_BLOCK_SIZE);
|
||||
break;
|
||||
|
||||
case MergeTreeData::Aggregating:
|
||||
merged_stream = ext::make_unique<AggregatingSortedBlockInputStream>(src_streams, data.getSortDescription(), DEFAULT_MERGE_BLOCK_SIZE);
|
||||
merged_stream = std::make_unique<AggregatingSortedBlockInputStream>(src_streams, data.getSortDescription(), DEFAULT_MERGE_BLOCK_SIZE);
|
||||
break;
|
||||
|
||||
default:
|
||||
|
@ -12,7 +12,7 @@
|
||||
|
||||
#include <DB/Core/Field.h>
|
||||
|
||||
#include <statdaemons/ext/memory.hpp>
|
||||
#include <memory>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -239,7 +239,7 @@ bool StorageDistributed::hasColumn(const String & column_name) const
|
||||
|
||||
void StorageDistributed::createDirectoryMonitor(const std::string & name)
|
||||
{
|
||||
directory_monitors.emplace(name, ext::make_unique<DirectoryMonitor>(*this, name));
|
||||
directory_monitors.emplace(name, std::make_unique<DirectoryMonitor>(*this, name));
|
||||
}
|
||||
|
||||
void StorageDistributed::createDirectoryMonitors()
|
||||
|
@ -2107,7 +2107,7 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params,
|
||||
auto zookeeper = getZooKeeper();
|
||||
const MergeTreeMergeBlocker merge_blocker{merger};
|
||||
const auto unreplicated_merge_blocker = unreplicated_merger ?
|
||||
ext::make_unique<MergeTreeMergeBlocker>(*unreplicated_merger) : nullptr;
|
||||
std::make_unique<MergeTreeMergeBlocker>(*unreplicated_merger) : nullptr;
|
||||
|
||||
LOG_DEBUG(log, "Doing ALTER");
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user