mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Changed boost:: to std:: for bind, ref/cref, thread, shared_ptr [#METR-2807].
This commit is contained in:
parent
afc4f7fcf0
commit
c2e4662ad7
@ -113,7 +113,7 @@ protected:
|
||||
void next()
|
||||
{
|
||||
ready.reset();
|
||||
pool.schedule(boost::bind(&AsynchronousBlockInputStream::calculate, this, current_memory_tracker));
|
||||
pool.schedule(std::bind(&AsynchronousBlockInputStream::calculate, this, current_memory_tracker));
|
||||
}
|
||||
|
||||
|
||||
|
@ -76,8 +76,8 @@ protected:
|
||||
for (size_t i = 0, size = many_data.size(); i < size; ++i)
|
||||
{
|
||||
many_data[i] = new AggregatedDataVariants;
|
||||
pool.schedule(boost::bind(&ParallelAggregatingBlockInputStream::calculate, this,
|
||||
boost::ref(children[i]), boost::ref(*many_data[i]), boost::ref(exceptions[i]), current_memory_tracker));
|
||||
pool.schedule(std::bind(&ParallelAggregatingBlockInputStream::calculate, this,
|
||||
std::ref(children[i]), std::ref(*many_data[i]), std::ref(exceptions[i]), current_memory_tracker));
|
||||
}
|
||||
pool.wait();
|
||||
|
||||
|
@ -51,7 +51,7 @@ private:
|
||||
swapBuffers();
|
||||
|
||||
/// Данные будут записываться в отельном потоке.
|
||||
pool.schedule(boost::bind(&AsynchronousWriteBuffer::thread, this));
|
||||
pool.schedule([this] { thread(); });
|
||||
}
|
||||
|
||||
public:
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <boost/thread.hpp>
|
||||
#include <thread>
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
@ -30,7 +30,7 @@ private:
|
||||
/// Периодичность обновления справочников, в секундах.
|
||||
int reload_period;
|
||||
|
||||
boost::thread reloading_thread;
|
||||
std::thread reloading_thread;
|
||||
Poco::Event destroy;
|
||||
|
||||
Logger * log;
|
||||
@ -139,7 +139,7 @@ public:
|
||||
log(&Logger::get("Dictionaries"))
|
||||
{
|
||||
reloadImpl();
|
||||
reloading_thread = boost::thread(&Dictionaries::reloadPeriodically, this);
|
||||
reloading_thread = std::thread([this] { reloadPeriodically(); });
|
||||
}
|
||||
|
||||
~Dictionaries()
|
||||
|
@ -57,7 +57,7 @@ private:
|
||||
Context & context;
|
||||
Settings settings;
|
||||
|
||||
boost::thread merge_thread;
|
||||
std::thread merge_thread;
|
||||
Poco::Event cancel_merge_thread;
|
||||
|
||||
Logger * log;
|
||||
|
@ -1,7 +1,7 @@
|
||||
#include <iostream>
|
||||
#include <stdlib.h>
|
||||
#include <vector>
|
||||
#include <boost/shared_ptr.hpp>
|
||||
#include <memory>
|
||||
|
||||
|
||||
struct C
|
||||
@ -218,7 +218,7 @@ struct IFactory
|
||||
virtual ~IFactory() {}
|
||||
};
|
||||
|
||||
typedef boost::shared_ptr<IFactory> FactoryPtr;
|
||||
typedef std::unique_ptr<IFactory> FactoryPtr;
|
||||
|
||||
|
||||
struct Factory1 : IFactory
|
||||
|
@ -109,8 +109,8 @@ int main(int argc, char ** argv)
|
||||
BlockOutputStreamPtr out1 = format_factory.getOutput("TabSeparated", ob1, out_sample);
|
||||
BlockOutputStreamPtr out2 = format_factory.getOutput("TabSeparated", ob2, out_sample);
|
||||
|
||||
boost::thread thr1(thread1, in1, out1, boost::ref(ob1));
|
||||
boost::thread thr2(thread2, in2, out2, boost::ref(ob2));
|
||||
std::thread thr1(std::bind(thread1, in1, out1, std::ref(ob1)));
|
||||
std::thread thr2(std::bind(thread2, in2, out2, std::ref(ob2)));
|
||||
|
||||
fork.run();
|
||||
|
||||
|
@ -84,11 +84,11 @@ int main(int argc, char ** argv)
|
||||
|
||||
boost::threadpool::pool pool(inputs.size() + forks.size());
|
||||
|
||||
pool.schedule(boost::bind(inputThread, inputs[0], out1, boost::ref(wb), boost::ref(mutex)));
|
||||
pool.schedule(boost::bind(inputThread, inputs[1], out2, boost::ref(wb), boost::ref(mutex)));
|
||||
pool.schedule(std::bind(inputThread, inputs[0], out1, std::ref(wb), std::ref(mutex)));
|
||||
pool.schedule(std::bind(inputThread, inputs[1], out2, std::ref(wb), std::ref(mutex)));
|
||||
|
||||
for (size_t i = 0; i < forks.size(); ++i)
|
||||
pool.schedule(boost::bind(forkThread, forks[i]));
|
||||
pool.schedule(std::bind(forkThread, forks[i]));
|
||||
|
||||
pool.wait();
|
||||
}
|
||||
|
@ -1,5 +1,3 @@
|
||||
#include <boost/bind.hpp>
|
||||
|
||||
#include <DB/Columns/ColumnArray.h>
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
|
||||
|
@ -1,5 +1,3 @@
|
||||
#include <boost/bind.hpp>
|
||||
|
||||
#include <DB/Columns/ColumnArray.h>
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
#include <DB/Columns/ColumnNested.h>
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <Poco/Util/Application.h>
|
||||
#include <Poco/Net/NetworkInterface.h>
|
||||
#include <boost/bind.hpp>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -17,8 +17,6 @@
|
||||
#include <Poco/FileStream.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <boost/bind.hpp>
|
||||
#include <boost/bind/placeholders.hpp>
|
||||
|
||||
|
||||
using namespace DB;
|
||||
|
@ -87,11 +87,11 @@ void SplittingAggregator::execute(BlockInputStreamPtr stream, ManyAggregatedData
|
||||
/// Параллельно вычисляем хэши и ключи.
|
||||
|
||||
for (size_t thread_no = 0; thread_no < threads; ++thread_no)
|
||||
pool.schedule(boost::bind(&SplittingAggregator::calculateHashesThread, this,
|
||||
boost::ref(block),
|
||||
pool.schedule(std::bind(&SplittingAggregator::calculateHashesThread, this,
|
||||
std::ref(block),
|
||||
rows * thread_no / threads,
|
||||
rows * (thread_no + 1) / threads,
|
||||
boost::ref(exceptions[thread_no]),
|
||||
std::ref(exceptions[thread_no]),
|
||||
current_memory_tracker));
|
||||
|
||||
pool.wait();
|
||||
@ -101,11 +101,11 @@ void SplittingAggregator::execute(BlockInputStreamPtr stream, ManyAggregatedData
|
||||
/// Параллельно агрегируем в независимые хэш-таблицы
|
||||
|
||||
for (size_t thread_no = 0; thread_no < threads; ++thread_no)
|
||||
pool.schedule(boost::bind(&SplittingAggregator::aggregateThread, this,
|
||||
boost::ref(block),
|
||||
boost::ref(*results[thread_no]),
|
||||
pool.schedule(std::bind(&SplittingAggregator::aggregateThread, this,
|
||||
std::ref(block),
|
||||
std::ref(*results[thread_no]),
|
||||
thread_no,
|
||||
boost::ref(exceptions[thread_no]),
|
||||
std::ref(exceptions[thread_no]),
|
||||
current_memory_tracker));
|
||||
|
||||
pool.wait();
|
||||
@ -131,11 +131,11 @@ void SplittingAggregator::convertToBlocks(ManyAggregatedDataVariants & data_vari
|
||||
/// Параллельно конвертируем в блоки.
|
||||
|
||||
for (size_t thread_no = 0; thread_no < threads; ++thread_no)
|
||||
pool.schedule(boost::bind(&SplittingAggregator::convertToBlockThread, this,
|
||||
boost::ref(*data_variants[thread_no]),
|
||||
boost::ref(blocks[thread_no]),
|
||||
pool.schedule(std::bind(&SplittingAggregator::convertToBlockThread, this,
|
||||
std::ref(*data_variants[thread_no]),
|
||||
std::ref(blocks[thread_no]),
|
||||
final,
|
||||
boost::ref(exceptions[thread_no]),
|
||||
std::ref(exceptions[thread_no]),
|
||||
current_memory_tracker));
|
||||
|
||||
pool.wait();
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <DB/Parsers/ASTIdentifier.h>
|
||||
#include <DB/Parsers/ASTNameTypePair.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <boost/bind.hpp>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <DB/Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Yandex/time2str.h>
|
||||
#include <Poco/Ext/ScopedTry.h>
|
||||
|
||||
#include <DB/Storages/MergeTree/MergeTreeData.h>
|
||||
#include <DB/Interpreters/ExpressionAnalyzer.h>
|
||||
#include <DB/Storages/MergeTree/MergeTreeReader.h>
|
||||
#include <DB/Storages/MergeTree/MergeTreeBlockInputStream.h>
|
||||
@ -12,7 +13,9 @@
|
||||
#include <DB/DataStreams/copyData.h>
|
||||
#include <DB/IO/WriteBufferFromFile.h>
|
||||
#include <DB/DataTypes/DataTypeDate.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <iomanip>
|
||||
|
||||
|
||||
|
||||
|
@ -232,7 +232,7 @@ StorageChunkMerger::StorageChunkMerger(
|
||||
context(context_), settings(context.getSettings()),
|
||||
log(&Logger::get("StorageChunkMerger")), shutdown_called(false)
|
||||
{
|
||||
merge_thread = boost::thread(&StorageChunkMerger::mergeThread, this);
|
||||
merge_thread = std::thread([this] { mergeThread(); });
|
||||
_table_column_name = "_table" + VirtualColumnUtils::chooseSuffix(getColumnsList(), "_table");
|
||||
}
|
||||
|
||||
|
@ -1,7 +1,7 @@
|
||||
#include <string.h>
|
||||
#include <iostream>
|
||||
#include <statdaemons/threadpool.hpp>
|
||||
#include <boost/bind.hpp>
|
||||
#include <functional>
|
||||
#include <Yandex/MultiVersion.h>
|
||||
|
||||
|
||||
@ -36,8 +36,8 @@ int main(int argc, char ** argv)
|
||||
boost::threadpool::pool tp(8);
|
||||
for (size_t i = 0; i < n; ++i)
|
||||
{
|
||||
tp.schedule(boost::bind(thread1, boost::ref(x), boost::ref(results[i])));
|
||||
tp.schedule(boost::bind(thread2, boost::ref(x), (rand() % 2) ? s1 : s2));
|
||||
tp.schedule(std::bind(thread1, std::ref(x), std::ref(results[i])));
|
||||
tp.schedule(std::bind(thread2, std::ref(x), (rand() % 2) ? s1 : s2));
|
||||
}
|
||||
tp.wait();
|
||||
|
||||
|
@ -175,7 +175,7 @@ private:
|
||||
static void processEvent(zhandle_t * zh, int type, int state, const char * path, void *watcherCtx);
|
||||
|
||||
template <class T>
|
||||
int32_t retry(const T & operation, size_t * attempt = nullptr)
|
||||
int32_t retry(T && operation, size_t * attempt = nullptr)
|
||||
{
|
||||
int32_t code = operation();
|
||||
if (attempt)
|
||||
|
@ -1,8 +1,9 @@
|
||||
#include <functional>
|
||||
#include <zkutil/ZooKeeper.h>
|
||||
#include <boost/make_shared.hpp>
|
||||
#include <Yandex/logger_useful.h>
|
||||
#include <DB/Common/ProfileEvents.h>
|
||||
#include <boost/bind.hpp>
|
||||
|
||||
|
||||
namespace zkutil
|
||||
{
|
||||
@ -176,7 +177,7 @@ Strings ZooKeeper::getChildren(
|
||||
int32_t ZooKeeper::tryGetChildren(const std::string & path, Strings & res,
|
||||
Stat * stat_, EventPtr watch)
|
||||
{
|
||||
int32_t code = retry(boost::bind(&ZooKeeper::getChildrenImpl, this, boost::ref(path), boost::ref(res), stat_, watch));
|
||||
int32_t code = retry(std::bind(&ZooKeeper::getChildrenImpl, this, std::ref(path), std::ref(res), stat_, watch));
|
||||
|
||||
if (!( code == ZOK ||
|
||||
code == ZNONODE))
|
||||
@ -234,14 +235,14 @@ int32_t ZooKeeper::tryCreate(const std::string & path, const std::string & data,
|
||||
|
||||
int32_t ZooKeeper::tryCreateWithRetries(const std::string & path, const std::string & data, int32_t mode, std::string & pathCreated, size_t* attempt)
|
||||
{
|
||||
return retry(boost::bind(&ZooKeeper::tryCreate, this, boost::ref(path), boost::ref(data), mode, boost::ref(pathCreated)), attempt);
|
||||
return retry([&path, &data, mode, &pathCreated, this] { return tryCreate(path, data, mode, pathCreated); });
|
||||
}
|
||||
|
||||
|
||||
void ZooKeeper::createIfNotExists(const std::string & path, const std::string & data)
|
||||
{
|
||||
std::string pathCreated;
|
||||
int32_t code = retry(boost::bind(&ZooKeeper::createImpl, this, boost::ref(path), boost::ref(data), zkutil::CreateMode::Persistent, boost::ref(pathCreated)));
|
||||
int32_t code = retry(std::bind(&ZooKeeper::createImpl, this, std::ref(path), std::ref(data), zkutil::CreateMode::Persistent, std::ref(pathCreated)));
|
||||
|
||||
if (code == ZOK || code == ZNODEEXISTS)
|
||||
return;
|
||||
@ -288,7 +289,7 @@ int32_t ZooKeeper::tryRemove(const std::string & path, int32_t version)
|
||||
|
||||
int32_t ZooKeeper::tryRemoveWithRetries(const std::string & path, int32_t version, size_t * attempt)
|
||||
{
|
||||
int32_t code = retry(boost::bind(&ZooKeeper::removeImpl, this, boost::ref(path), version), attempt);
|
||||
int32_t code = retry(std::bind(&ZooKeeper::removeImpl, this, std::ref(path), version), attempt);
|
||||
if (!( code == ZOK ||
|
||||
code == ZNONODE ||
|
||||
code == ZBADVERSION ||
|
||||
@ -316,7 +317,7 @@ int32_t ZooKeeper::existsImpl(const std::string & path, Stat * stat_, EventPtr w
|
||||
|
||||
bool ZooKeeper::exists(const std::string & path, Stat * stat_, EventPtr watch)
|
||||
{
|
||||
int32_t code = retry(boost::bind(&ZooKeeper::existsImpl, this, path, stat_, watch));
|
||||
int32_t code = retry(std::bind(&ZooKeeper::existsImpl, this, path, stat_, watch));
|
||||
|
||||
if (!( code == ZOK ||
|
||||
code == ZNONODE))
|
||||
@ -360,7 +361,7 @@ std::string ZooKeeper::get(const std::string & path, Stat * stat, EventPtr watch
|
||||
|
||||
bool ZooKeeper::tryGet(const std::string & path, std::string & res, Stat * stat_, EventPtr watch)
|
||||
{
|
||||
int32_t code = retry(boost::bind(&ZooKeeper::getImpl, this, boost::ref(path), boost::ref(res), stat_, watch));
|
||||
int32_t code = retry(std::bind(&ZooKeeper::getImpl, this, std::ref(path), std::ref(res), stat_, watch));
|
||||
|
||||
if (!( code == ZOK ||
|
||||
code == ZNONODE))
|
||||
@ -455,7 +456,7 @@ int32_t ZooKeeper::tryMulti(const Ops & ops_, OpResultsPtr * out_results_)
|
||||
|
||||
int32_t ZooKeeper::tryMultiWithRetries(const Ops & ops, OpResultsPtr * out_results, size_t * attempt)
|
||||
{
|
||||
int32_t code = retry(boost::bind(&ZooKeeper::multiImpl, this, boost::ref(ops), out_results), attempt);
|
||||
int32_t code = retry(std::bind(&ZooKeeper::multiImpl, this, std::ref(ops), out_results), attempt);
|
||||
if (!(code == ZOK ||
|
||||
code == ZNONODE ||
|
||||
code == ZNODEEXISTS ||
|
||||
|
@ -187,7 +187,7 @@ int mainImpl(int argc, char ** argv)
|
||||
Stopwatch watch;
|
||||
|
||||
for (size_t i = 0; i < threads; ++i)
|
||||
pool.schedule(boost::bind(thread, fd, mode, min_offset, max_offset, block_size, count, boost::ref(exceptions[i])));
|
||||
pool.schedule(std::bind(thread, fd, mode, min_offset, max_offset, block_size, count, std::ref(exceptions[i])));
|
||||
pool.wait();
|
||||
|
||||
fsync(fd);
|
||||
|
@ -278,7 +278,7 @@ int mainImpl(int argc, char ** argv)
|
||||
Stopwatch watch;
|
||||
|
||||
for (size_t i = 0; i < threads_count; ++i)
|
||||
pool.schedule(boost::bind(thread, fd, mode, min_offset, max_offset, block_size, buffers_count, count, boost::ref(exceptions[i])));
|
||||
pool.schedule(std::bind(thread, fd, mode, min_offset, max_offset, block_size, buffers_count, count, std::ref(exceptions[i])));
|
||||
pool.wait();
|
||||
|
||||
watch.stop();
|
||||
|
Loading…
Reference in New Issue
Block a user