Using std::make_unique [#METR-2807].

This commit is contained in:
Alexey Milovidov 2015-02-11 00:10:58 +03:00
parent 2d37140ee1
commit a9eb28c07d
17 changed files with 37 additions and 37 deletions

View File

@ -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, если запрос отправлен, а ещё не выполнен.

View File

@ -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);
}

View File

@ -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(); }

View File

@ -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(); }

View File

@ -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(); }

View File

@ -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)};

View File

@ -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)
{

View File

@ -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>

View File

@ -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;

View File

@ -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

View File

@ -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>

View File

@ -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>

View File

@ -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{

View File

@ -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", "::");

View File

@ -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:

View File

@ -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()

View File

@ -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");