mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Using std::shared_ptr for data types [#METR-21503].
This commit is contained in:
parent
f9b2cd5ab6
commit
6863e72f3d
@ -24,12 +24,15 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
/// Поток блоков читающих из таблицы и ее имя
|
||||
typedef std::pair<BlockInputStreamPtr, std::string> ExternalTableData;
|
||||
using ExternalTableData = std::pair<BlockInputStreamPtr, std::string>;
|
||||
/// Вектор пар, описывающих таблицы
|
||||
typedef std::vector<ExternalTableData> ExternalTablesData;
|
||||
using ExternalTablesData = std::vector<ExternalTableData>;
|
||||
|
||||
class Connection;
|
||||
|
||||
using ConnectionPtr = std::shared_ptr<Connection>;
|
||||
using Connections = std::vector<ConnectionPtr>;
|
||||
|
||||
|
||||
/** Соединение с сервером БД для использования в клиенте.
|
||||
@ -270,8 +273,4 @@ private:
|
||||
void initBlockInput();
|
||||
};
|
||||
|
||||
|
||||
typedef SharedPtr<Connection> ConnectionPtr;
|
||||
typedef std::vector<ConnectionPtr> Connections;
|
||||
|
||||
}
|
||||
|
@ -25,7 +25,7 @@ using Poco::SharedPtr;
|
||||
class IConnectionPool : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
typedef PoolBase<Connection>::Entry Entry;
|
||||
using Entry = PoolBase<Connection>::Entry;
|
||||
|
||||
public:
|
||||
virtual ~IConnectionPool() {}
|
||||
@ -56,9 +56,9 @@ protected:
|
||||
}
|
||||
};
|
||||
|
||||
typedef SharedPtr<IConnectionPool> ConnectionPoolPtr;
|
||||
typedef std::vector<ConnectionPoolPtr> ConnectionPools;
|
||||
typedef SharedPtr<ConnectionPools> ConnectionPoolsPtr;
|
||||
using ConnectionPoolPtr = std::shared_ptr<IConnectionPool>;
|
||||
using ConnectionPools = std::vector<ConnectionPoolPtr>;
|
||||
using ConnectionPoolsPtr = std::shared_ptr<ConnectionPools>;
|
||||
|
||||
|
||||
/** Обычный пул соединений, без отказоустойчивости.
|
||||
@ -66,8 +66,8 @@ typedef SharedPtr<ConnectionPools> ConnectionPoolsPtr;
|
||||
class ConnectionPool : public PoolBase<Connection>, public IConnectionPool
|
||||
{
|
||||
public:
|
||||
typedef IConnectionPool::Entry Entry;
|
||||
typedef PoolBase<Connection> Base;
|
||||
using Entry = IConnectionPool::Entry;
|
||||
using Base = PoolBase<Connection>;
|
||||
|
||||
ConnectionPool(unsigned max_connections_,
|
||||
const String & host_, UInt16 port_,
|
||||
@ -112,7 +112,7 @@ protected:
|
||||
/** Создает новый объект для помещения в пул. */
|
||||
ConnectionPtr allocObject() override
|
||||
{
|
||||
return new Connection(
|
||||
return std::make_shared<Connection>(
|
||||
host, port, resolved_address,
|
||||
default_database, user, password,
|
||||
client_name, compression,
|
||||
|
@ -4,7 +4,7 @@
|
||||
#include <DB/Client/Connection.h>
|
||||
#include <DB/Client/ConnectionPool.h>
|
||||
#include <Poco/ScopedLock.h>
|
||||
#include <Poco/Mutex.h>
|
||||
#include <mutex>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -159,7 +159,7 @@ private:
|
||||
|
||||
/// Мьютекс для того, чтобы функция sendCancel могла выполняться безопасно
|
||||
/// в отдельном потоке.
|
||||
mutable Poco::FastMutex cancel_mutex;
|
||||
mutable std::mutex cancel_mutex;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -136,7 +136,7 @@ public:
|
||||
|
||||
ColumnPtr cloneEmpty() const override
|
||||
{
|
||||
return std::make_shared<ColumnAggregateFunction>(holder->func, Arenas(1, new Arena));
|
||||
return std::make_shared<ColumnAggregateFunction>(holder->func, Arenas(1, std::make_shared<Arena>()));
|
||||
};
|
||||
|
||||
Field operator[](size_t n) const override
|
||||
@ -182,7 +182,7 @@ public:
|
||||
Arena & createOrGetArena()
|
||||
{
|
||||
if (unlikely(arenas.empty()))
|
||||
arenas.emplace_back(new Arena);
|
||||
arenas.emplace_back(std::make_shared<Arena>());
|
||||
return *arenas.back().get();
|
||||
}
|
||||
|
||||
|
@ -4,7 +4,6 @@
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <Poco/SharedPtr.h>
|
||||
#include <common/likely.h>
|
||||
#include <DB/Core/Defines.h>
|
||||
#include <DB/Common/ProfileEvents.h>
|
||||
@ -164,8 +163,8 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
typedef Poco::SharedPtr<Arena> ArenaPtr;
|
||||
typedef std::vector<ArenaPtr> Arenas;
|
||||
using ArenaPtr = std::shared_ptr<Arena>;
|
||||
using Arenas = std::vector<ArenaPtr>;
|
||||
|
||||
|
||||
}
|
||||
|
@ -18,7 +18,7 @@ class ConcurrentBoundedQueue
|
||||
private:
|
||||
size_t max_fill;
|
||||
std::queue<T> queue;
|
||||
Poco::Mutex mutex;
|
||||
Poco::FastMutex mutex;
|
||||
Poco::Semaphore fill_count;
|
||||
Poco::Semaphore empty_count;
|
||||
|
||||
@ -30,7 +30,7 @@ public:
|
||||
{
|
||||
empty_count.wait();
|
||||
{
|
||||
Poco::ScopedLock<Poco::Mutex> lock(mutex);
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
queue.push(x);
|
||||
}
|
||||
fill_count.set();
|
||||
@ -40,7 +40,7 @@ public:
|
||||
{
|
||||
fill_count.wait();
|
||||
{
|
||||
Poco::ScopedLock<Poco::Mutex> lock(mutex);
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
x = queue.front();
|
||||
queue.pop();
|
||||
}
|
||||
@ -52,7 +52,7 @@ public:
|
||||
if (empty_count.tryWait(milliseconds))
|
||||
{
|
||||
{
|
||||
Poco::ScopedLock<Poco::Mutex> lock(mutex);
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
queue.push(x);
|
||||
}
|
||||
fill_count.set();
|
||||
@ -66,7 +66,7 @@ public:
|
||||
if (fill_count.tryWait(milliseconds))
|
||||
{
|
||||
{
|
||||
Poco::ScopedLock<Poco::Mutex> lock(mutex);
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
x = queue.front();
|
||||
queue.pop();
|
||||
}
|
||||
@ -78,7 +78,7 @@ public:
|
||||
|
||||
size_t size()
|
||||
{
|
||||
Poco::ScopedLock<Poco::Mutex> lock(mutex);
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
return queue.size();
|
||||
}
|
||||
|
||||
@ -87,7 +87,7 @@ public:
|
||||
while (fill_count.tryWait(0))
|
||||
{
|
||||
{
|
||||
Poco::ScopedLock<Poco::Mutex> lock(mutex);
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
queue.pop();
|
||||
}
|
||||
empty_count.set();
|
||||
|
@ -8,7 +8,7 @@
|
||||
|
||||
#include <Poco/File.h>
|
||||
#include <Poco/Exception.h>
|
||||
#include <Poco/Mutex.h>
|
||||
#include <mutex>
|
||||
#include <Poco/ScopedLock.h>
|
||||
|
||||
#include <DB/Common/Exception.h>
|
||||
@ -44,7 +44,7 @@ public:
|
||||
template <typename Callback>
|
||||
Int64 add(Int64 delta, Callback && locked_callback, bool create_if_need = false)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
Int64 res = -1;
|
||||
|
||||
@ -177,7 +177,7 @@ public:
|
||||
|
||||
private:
|
||||
std::string path;
|
||||
Poco::FastMutex mutex;
|
||||
std::mutex mutex;
|
||||
};
|
||||
|
||||
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <memory>
|
||||
#include <chrono>
|
||||
#include <Poco/ScopedLock.h>
|
||||
#include <Poco/Mutex.h>
|
||||
#include <mutex>
|
||||
#include <DB/Common/Exception.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
@ -46,7 +46,7 @@ public:
|
||||
|
||||
MappedPtr get(const Key & key)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
auto it = cells.find(key);
|
||||
if (it == cells.end())
|
||||
@ -67,7 +67,7 @@ public:
|
||||
|
||||
void set(const Key & key, MappedPtr mapped)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
auto res = cells.emplace(std::piecewise_construct,
|
||||
std::forward_as_tuple(key),
|
||||
@ -96,26 +96,26 @@ public:
|
||||
|
||||
void getStats(size_t & out_hits, size_t & out_misses) const
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
out_hits = hits;
|
||||
out_misses = misses;
|
||||
}
|
||||
|
||||
size_t weight() const
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
return current_size;
|
||||
}
|
||||
|
||||
size_t count() const
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
return cells.size();
|
||||
}
|
||||
|
||||
void reset()
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
queue.clear();
|
||||
cells.clear();
|
||||
current_size = 0;
|
||||
@ -158,7 +158,7 @@ private:
|
||||
const size_t max_size;
|
||||
const Delay expiration_delay;
|
||||
|
||||
mutable Poco::FastMutex mutex;
|
||||
mutable std::mutex mutex;
|
||||
size_t hits = 0;
|
||||
size_t misses = 0;
|
||||
|
||||
|
@ -1,8 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
#include <Poco/Mutex.h>
|
||||
#include <Poco/Condition.h>
|
||||
#include <mutex>
|
||||
#include <condition_variable>
|
||||
#include <Poco/Timespan.h>
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
@ -28,26 +27,26 @@ template <typename TObject>
|
||||
class PoolBase : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
typedef TObject Object;
|
||||
typedef Poco::SharedPtr<Object> ObjectPtr;
|
||||
typedef Poco::SharedPtr<PoolBase<TObject> > Ptr;
|
||||
using Object = TObject;
|
||||
using ObjectPtr = std::shared_ptr<Object>;
|
||||
using Ptr = std::shared_ptr<PoolBase<TObject>>;
|
||||
|
||||
private:
|
||||
|
||||
/** Объект с флагом, используется ли он сейчас. */
|
||||
struct PooledObject
|
||||
{
|
||||
PooledObject(Poco::Condition & available_, ObjectPtr object_)
|
||||
PooledObject(std::condition_variable & available_, ObjectPtr object_)
|
||||
: object(object_), available(available_)
|
||||
{
|
||||
}
|
||||
|
||||
ObjectPtr object;
|
||||
bool in_use = false;
|
||||
Poco::Condition & available;
|
||||
std::condition_variable & available;
|
||||
};
|
||||
|
||||
typedef std::vector<Poco::SharedPtr<PooledObject> > Objects;
|
||||
using Objects = std::vector<std::shared_ptr<PooledObject>>;
|
||||
|
||||
/** Помощник, который устанавливает флаг использования объекта, а в деструкторе - снимает,
|
||||
* а также уведомляет о событии с помощью condvar-а.
|
||||
@ -55,7 +54,7 @@ private:
|
||||
struct PoolEntryHelper
|
||||
{
|
||||
PoolEntryHelper(PooledObject & data_) : data(data_) { data.in_use = true; }
|
||||
~PoolEntryHelper() { data.in_use = false; data.available.signal(); }
|
||||
~PoolEntryHelper() { data.in_use = false; data.available.notify_one(); }
|
||||
|
||||
PooledObject & data;
|
||||
};
|
||||
@ -88,9 +87,9 @@ public:
|
||||
bool isNull() const { return data.isNull(); }
|
||||
|
||||
private:
|
||||
Poco::SharedPtr<PoolEntryHelper> data;
|
||||
std::shared_ptr<PoolEntryHelper> data;
|
||||
|
||||
Entry(PooledObject & object) : data(new PoolEntryHelper(object)) {}
|
||||
Entry(PooledObject & object) : data(std::make_shared<PoolEntryHelper>(object)) {}
|
||||
};
|
||||
|
||||
virtual ~PoolBase() {}
|
||||
@ -98,36 +97,36 @@ public:
|
||||
/** Выделяет объект для работы. При timeout < 0 таймаут бесконечный. */
|
||||
Entry get(Poco::Timespan::TimeDiff timeout)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
|
||||
while (true)
|
||||
{
|
||||
for (typename Objects::iterator it = items.begin(); it != items.end(); it++)
|
||||
if (!(*it)->in_use)
|
||||
return Entry(**it);
|
||||
for (auto & item : items)
|
||||
if (!item->in_use)
|
||||
return Entry(*item);
|
||||
|
||||
if (items.size() < max_items)
|
||||
{
|
||||
ObjectPtr object = allocObject();
|
||||
items.push_back(new PooledObject(available, object));
|
||||
items.emplace_back(std::make_shared<PooledObject>(available, object));
|
||||
return Entry(*items.back());
|
||||
}
|
||||
|
||||
LOG_INFO(log, "No free connections in pool. Waiting.");
|
||||
|
||||
if (timeout < 0)
|
||||
available.wait(mutex);
|
||||
available.wait(lock);
|
||||
else
|
||||
available.wait(mutex, timeout);
|
||||
available.wait_for(lock, std::chrono::microseconds(timeout));
|
||||
}
|
||||
}
|
||||
|
||||
void reserve(size_t count)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
while (items.size() < count)
|
||||
items.push_back(new PooledObject(available, allocObject()));
|
||||
items.emplace_back(std::make_shared<PooledObject>(available, allocObject()));
|
||||
}
|
||||
|
||||
private:
|
||||
@ -138,8 +137,8 @@ private:
|
||||
Objects items;
|
||||
|
||||
/** Блокировка для доступа к пулу. */
|
||||
Poco::FastMutex mutex;
|
||||
Poco::Condition available;
|
||||
std::mutex mutex;
|
||||
std::condition_variable available;
|
||||
|
||||
protected:
|
||||
|
||||
|
@ -78,10 +78,10 @@ template <typename TNestedPool>
|
||||
class PoolWithFailoverBase : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
typedef TNestedPool NestedPool;
|
||||
typedef Poco::SharedPtr<NestedPool> NestedPoolPtr;
|
||||
typedef typename NestedPool::Entry Entry;
|
||||
typedef std::vector<NestedPoolPtr> NestedPools;
|
||||
using NestedPool = TNestedPool;
|
||||
using NestedPoolPtr = std::shared_ptr<NestedPool>;
|
||||
using Entry = typename NestedPool::Entry;
|
||||
using NestedPools = std::vector<NestedPoolPtr>;
|
||||
|
||||
virtual ~PoolWithFailoverBase() {}
|
||||
|
||||
@ -208,7 +208,7 @@ protected:
|
||||
states.reserve(this->size());
|
||||
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
for (auto & pool : *this)
|
||||
pool.randomize();
|
||||
@ -255,7 +255,7 @@ protected:
|
||||
/// Время, когда последний раз уменьшался счётчик ошибок.
|
||||
time_t last_decrease_time = 0;
|
||||
time_t decrease_error_period;
|
||||
Poco::FastMutex mutex;
|
||||
std::mutex mutex;
|
||||
};
|
||||
|
||||
PoolsWithErrorCount nested_pools;
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <time.h>
|
||||
#include <Poco/Mutex.h>
|
||||
#include <mutex>
|
||||
#include <Poco/ScopedLock.h>
|
||||
#include <common/Common.h>
|
||||
|
||||
@ -59,7 +59,7 @@ public:
|
||||
*/
|
||||
bool lockTestAndRestart(double seconds)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
if (elapsedSeconds() >= seconds)
|
||||
{
|
||||
@ -71,5 +71,5 @@ public:
|
||||
}
|
||||
|
||||
private:
|
||||
Poco::FastMutex mutex;
|
||||
std::mutex mutex;
|
||||
};
|
||||
|
@ -6,12 +6,10 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Позволяет из одного источника сделать несколько.
|
||||
* Используется для однопроходного выполнения сразу нескольких запросов.
|
||||
*
|
||||
*
|
||||
* Несколько полученных источников должны читаться из разных потоков!
|
||||
* Расходует O(1) оперативки (не буферизует все данные).
|
||||
* Для этого, чтения из разных полученных источников синхронизируются:
|
||||
@ -25,7 +23,7 @@ public:
|
||||
/// Создать источник. Вызывайте функцию столько раз, сколько размноженных источников вам нужно.
|
||||
BlockInputStreamPtr createInput()
|
||||
{
|
||||
destinations.push_back(new QueueBlockIOStream(1));
|
||||
destinations.emplace_back(new QueueBlockIOStream(1));
|
||||
return destinations.back();
|
||||
}
|
||||
|
||||
@ -65,12 +63,12 @@ private:
|
||||
* Сделаны на основе очереди небольшой длины.
|
||||
* Блок из source кладётся в каждую очередь.
|
||||
*/
|
||||
typedef SharedPtr<QueueBlockIOStream> Destination;
|
||||
typedef std::list<Destination> Destinations;
|
||||
using Destination = Poco::SharedPtr<QueueBlockIOStream>;
|
||||
using Destinations = std::list<Destination>;
|
||||
Destinations destinations;
|
||||
};
|
||||
|
||||
typedef SharedPtr<ForkBlockInputStreams> ForkPtr;
|
||||
typedef std::vector<ForkPtr> Forks;
|
||||
using ForkPtr = std::shared_ptr<ForkBlockInputStreams>;
|
||||
using Forks = std::vector<ForkPtr>;
|
||||
|
||||
}
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/Mutex.h>
|
||||
#include <mutex>
|
||||
|
||||
#include <DB/Common/OptimizedRegularExpression.h>
|
||||
#include <memory>
|
||||
|
@ -609,7 +609,7 @@ struct AggregatedDataVariants : private boost::noncopyable
|
||||
};
|
||||
Type type = Type::EMPTY;
|
||||
|
||||
AggregatedDataVariants() : aggregates_pools(1, new Arena), aggregates_pool(aggregates_pools.back().get()) {}
|
||||
AggregatedDataVariants() : aggregates_pools(1, std::make_shared<Arena>()), aggregates_pool(aggregates_pools.back().get()) {}
|
||||
bool empty() const { return type == Type::EMPTY; }
|
||||
void invalidate() { type = Type::EMPTY; }
|
||||
|
||||
|
@ -65,7 +65,7 @@ public:
|
||||
{
|
||||
public:
|
||||
bool isLocal() const { return !local_addresses.empty(); }
|
||||
bool hasRemoteConnections() const { return !pool.isNull(); }
|
||||
bool hasRemoteConnections() const { return pool.get() != nullptr; }
|
||||
size_t getLocalNodeCount() const { return local_addresses.size(); }
|
||||
|
||||
public:
|
||||
|
@ -74,7 +74,7 @@ protected:
|
||||
std::atomic<bool> is_cancelled {false};
|
||||
};
|
||||
|
||||
typedef Poco::SharedPtr<InterserverIOEndpoint> InterserverIOEndpointPtr;
|
||||
using InterserverIOEndpointPtr = std::shared_ptr<InterserverIOEndpoint>;
|
||||
|
||||
|
||||
/** Сюда можно зарегистрировать сервис, обрататывающий запросы от других серверов.
|
||||
@ -85,7 +85,7 @@ class InterserverIOHandler
|
||||
public:
|
||||
void addEndpoint(const String & name, InterserverIOEndpointPtr endpoint)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
if (endpoint_map.count(name))
|
||||
throw Exception("Duplicate interserver IO endpoint: " + name, ErrorCodes::DUPLICATE_INTERSERVER_IO_ENDPOINT);
|
||||
endpoint_map[name] = endpoint;
|
||||
@ -93,7 +93,7 @@ public:
|
||||
|
||||
void removeEndpoint(const String & name)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
if (!endpoint_map.count(name))
|
||||
throw Exception("No interserver IO endpoint named " + name, ErrorCodes::NO_SUCH_INTERSERVER_IO_ENDPOINT);
|
||||
endpoint_map.erase(name);
|
||||
@ -101,7 +101,7 @@ public:
|
||||
|
||||
InterserverIOEndpointPtr getEndpoint(const String & name)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
if (!endpoint_map.count(name))
|
||||
throw Exception("No interserver IO endpoint named " + name, ErrorCodes::NO_SUCH_INTERSERVER_IO_ENDPOINT);
|
||||
return endpoint_map[name];
|
||||
@ -111,7 +111,7 @@ private:
|
||||
typedef std::map<String, InterserverIOEndpointPtr> EndpointMap;
|
||||
|
||||
EndpointMap endpoint_map;
|
||||
Poco::FastMutex mutex;
|
||||
std::mutex mutex;
|
||||
};
|
||||
|
||||
/// В конструкторе вызывает addEndpoint, в деструкторе - removeEndpoint.
|
||||
@ -151,6 +151,6 @@ private:
|
||||
InterserverIOHandler & handler;
|
||||
};
|
||||
|
||||
typedef Poco::SharedPtr<InterserverIOEndpointHolder> InterserverIOEndpointHolderPtr;
|
||||
using InterserverIOEndpointHolderPtr = std::shared_ptr<InterserverIOEndpointHolder>;
|
||||
|
||||
}
|
||||
|
@ -3,7 +3,7 @@
|
||||
#include <map>
|
||||
#include <list>
|
||||
#include <memory>
|
||||
#include <Poco/Mutex.h>
|
||||
#include <mutex>
|
||||
#include <Poco/Condition.h>
|
||||
#include <Poco/Net/IPAddress.h>
|
||||
#include <DB/Common/Stopwatch.h>
|
||||
@ -162,7 +162,7 @@ public:
|
||||
using UserToQueries = std::unordered_map<String, ProcessListForUser>;
|
||||
|
||||
private:
|
||||
mutable Poco::FastMutex mutex;
|
||||
mutable std::mutex mutex;
|
||||
mutable Poco::Condition have_space; /// Количество одновременно выполняющихся запросов стало меньше максимального.
|
||||
|
||||
Container cont;
|
||||
@ -192,7 +192,7 @@ public:
|
||||
/// Получить текущее состояние списка запросов.
|
||||
Info getInfo() const
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
Info res;
|
||||
res.reserve(cur_size);
|
||||
@ -204,7 +204,7 @@ public:
|
||||
|
||||
void setMaxSize(size_t max_size_)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
max_size = max_size_;
|
||||
}
|
||||
|
||||
|
@ -171,7 +171,7 @@ struct Quota
|
||||
/// Максимальные и накопленные значения для разных ключей.
|
||||
/// Для всех ключей максимальные значения одинаковы и взяты из max.
|
||||
Container quota_for_keys;
|
||||
Poco::FastMutex mutex;
|
||||
std::mutex mutex;
|
||||
|
||||
bool is_keyed;
|
||||
bool keyed_by_ip;
|
||||
|
@ -6,7 +6,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
typedef std::pair<const char *, const char *> StringRange;
|
||||
typedef Poco::SharedPtr<String> StringPtr;
|
||||
using StringRange = std::pair<const char *, const char *>;
|
||||
using StringPtr = std::shared_ptr<String>;
|
||||
|
||||
}
|
||||
|
@ -136,15 +136,15 @@ private:
|
||||
{
|
||||
const auto pool_factory = [this, &name] (const std::string & host, const UInt16 port,
|
||||
const std::string & user, const std::string & password) {
|
||||
return new ConnectionPool{
|
||||
return std::make_shared<ConnectionPool>(
|
||||
1, host, port, "",
|
||||
user, password,
|
||||
storage.getName() + '_' + name};
|
||||
storage.getName() + '_' + name);
|
||||
};
|
||||
|
||||
auto pools = createPoolsForAddresses(name, pool_factory);
|
||||
|
||||
return pools.size() == 1 ? pools.front() : new ConnectionPoolWithFailover(pools, LoadBalancing::RANDOM);
|
||||
return pools.size() == 1 ? pools.front() : std::make_shared<ConnectionPoolWithFailover>(pools, LoadBalancing::RANDOM);
|
||||
}
|
||||
|
||||
bool findFiles()
|
||||
|
@ -30,14 +30,14 @@ class Context;
|
||||
class IBlockInputStream;
|
||||
class IBlockOutputStream;
|
||||
|
||||
typedef SharedPtr<IBlockOutputStream> BlockOutputStreamPtr;
|
||||
typedef SharedPtr<IBlockInputStream> BlockInputStreamPtr;
|
||||
typedef std::vector<BlockInputStreamPtr> BlockInputStreams;
|
||||
using BlockOutputStreamPtr = SharedPtr<IBlockOutputStream>;
|
||||
using BlockInputStreamPtr = SharedPtr<IBlockInputStream>;
|
||||
using BlockInputStreams = std::vector<BlockInputStreamPtr>;
|
||||
|
||||
|
||||
class IStorage;
|
||||
|
||||
typedef std::shared_ptr<IStorage> StoragePtr;
|
||||
using StoragePtr = std::shared_ptr<IStorage>;
|
||||
|
||||
|
||||
/** Хранилище. Отвечает за:
|
||||
@ -84,6 +84,7 @@ public:
|
||||
std::experimental::optional<Poco::ScopedReadRWLock> data_lock;
|
||||
std::experimental::optional<Poco::ScopedReadRWLock> structure_lock;
|
||||
|
||||
public:
|
||||
TableStructureReadLock(StoragePtr storage_, bool lock_structure, bool lock_data) : storage(storage_)
|
||||
{
|
||||
if (lock_data)
|
||||
@ -93,8 +94,8 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
typedef Poco::SharedPtr<TableStructureReadLock> TableStructureReadLockPtr;
|
||||
typedef std::vector<TableStructureReadLockPtr> TableStructureReadLocks;
|
||||
using TableStructureReadLockPtr = std::shared_ptr<TableStructureReadLock>;
|
||||
using TableStructureReadLocks = std::vector<TableStructureReadLockPtr>;
|
||||
|
||||
/** Не дает изменять структуру или имя таблицы.
|
||||
* Если в рамках этого лока будут изменены данные в таблице, нужно указать will_modify_data=true.
|
||||
@ -105,15 +106,15 @@ public:
|
||||
*/
|
||||
TableStructureReadLockPtr lockStructure(bool will_modify_data)
|
||||
{
|
||||
TableStructureReadLockPtr res = new TableStructureReadLock(thisPtr(), true, will_modify_data);
|
||||
TableStructureReadLockPtr res = std::make_shared<TableStructureReadLock>(thisPtr(), true, will_modify_data);
|
||||
if (is_dropped)
|
||||
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);
|
||||
return res;
|
||||
}
|
||||
|
||||
typedef std::unique_ptr<Poco::ScopedWriteRWLock> TableStructureWriteLockPtr;
|
||||
typedef std::unique_ptr<Poco::ScopedWriteRWLock> TableDataWriteLockPtr;
|
||||
typedef std::pair<TableDataWriteLockPtr, TableStructureWriteLockPtr> TableFullWriteLockPtr;
|
||||
using TableStructureWriteLockPtr = std::unique_ptr<Poco::ScopedWriteRWLock>;
|
||||
using TableDataWriteLockPtr = std::unique_ptr<Poco::ScopedWriteRWLock>;
|
||||
using TableFullWriteLockPtr = std::pair<TableDataWriteLockPtr, TableStructureWriteLockPtr>;
|
||||
|
||||
/** Не дает читать структуру таблицы. Берется для ALTER, RENAME и DROP.
|
||||
*/
|
||||
|
@ -1,5 +1,5 @@
|
||||
#pragma once
|
||||
#include <Poco/Mutex.h>
|
||||
#include <mutex>
|
||||
#include <Poco/RegularExpression.h>
|
||||
#include <common/DateLUT.h>
|
||||
#include <DB/Core/Types.h>
|
||||
@ -79,7 +79,7 @@ public:
|
||||
private:
|
||||
typedef std::set<Part> Parts;
|
||||
|
||||
mutable Poco::Mutex mutex;
|
||||
mutable std::mutex mutex;
|
||||
Parts parts;
|
||||
|
||||
void addImpl(const String & name);
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <map>
|
||||
#include <list>
|
||||
#include <condition_variable>
|
||||
#include <Poco/Mutex.h>
|
||||
#include <mutex>
|
||||
#include <Poco/RWLock.h>
|
||||
#include <Poco/Event.h>
|
||||
#include <Poco/SharedPtr.h>
|
||||
@ -26,7 +26,7 @@ namespace DB
|
||||
class BackgroundProcessingPool
|
||||
{
|
||||
public:
|
||||
typedef std::map<String, int> Counters;
|
||||
using Counters = std::map<String, int>;
|
||||
|
||||
/** Используется изнутри задачи. Позволяет инкрементировать какие-нибудь счетчики.
|
||||
* После завершения задачи, все изменения откатятся.
|
||||
@ -53,7 +53,7 @@ public:
|
||||
};
|
||||
|
||||
/// Возвращает true, если что-то получилось сделать. В таком случае поток не будет спать перед следующим вызовом.
|
||||
typedef std::function<bool (Context & context)> Task;
|
||||
using Task = std::function<bool (Context & context)>;
|
||||
|
||||
|
||||
class TaskInfo
|
||||
@ -78,7 +78,7 @@ public:
|
||||
TaskInfo(BackgroundProcessingPool & pool_, const Task & function_) : pool(pool_), function(function_) {}
|
||||
};
|
||||
|
||||
typedef std::shared_ptr<TaskInfo> TaskHandle;
|
||||
using TaskHandle = std::shared_ptr<TaskInfo>;
|
||||
|
||||
|
||||
BackgroundProcessingPool(int size_);
|
||||
@ -96,8 +96,8 @@ public:
|
||||
~BackgroundProcessingPool();
|
||||
|
||||
private:
|
||||
typedef std::list<TaskHandle> Tasks;
|
||||
typedef std::vector<std::thread> Threads;
|
||||
using Tasks = std::list<TaskHandle>;
|
||||
using Threads = std::vector<std::thread>;
|
||||
|
||||
const size_t size;
|
||||
static constexpr double sleep_seconds = 10;
|
||||
@ -116,6 +116,6 @@ private:
|
||||
void threadFunction();
|
||||
};
|
||||
|
||||
typedef Poco::SharedPtr<BackgroundProcessingPool> BackgroundProcessingPoolPtr;
|
||||
using BackgroundProcessingPoolPtr = std::shared_ptr<BackgroundProcessingPool>;
|
||||
|
||||
}
|
||||
|
@ -1,6 +1,8 @@
|
||||
#pragma once
|
||||
#include <Poco/Mutex.h>
|
||||
|
||||
#include <mutex>
|
||||
#include <sys/statvfs.h>
|
||||
#include <memory>
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <common/logger_useful.h>
|
||||
#include <DB/Common/Exception.h>
|
||||
@ -28,13 +30,12 @@ class DiskSpaceMonitor
|
||||
public:
|
||||
class Reservation : private boost::noncopyable
|
||||
{
|
||||
friend class DiskSpaceMonitor;
|
||||
public:
|
||||
~Reservation()
|
||||
{
|
||||
try
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(DiskSpaceMonitor::mutex);
|
||||
std::lock_guard<std::mutex> lock(DiskSpaceMonitor::mutex);
|
||||
if (DiskSpaceMonitor::reserved_bytes < size)
|
||||
{
|
||||
DiskSpaceMonitor::reserved_bytes = 0;
|
||||
@ -63,7 +64,7 @@ public:
|
||||
/// Изменить количество зарезервированного места. При увеличении не делается проверка, что места достаточно.
|
||||
void update(size_t new_size)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(DiskSpaceMonitor::mutex);
|
||||
std::lock_guard<std::mutex> lock(DiskSpaceMonitor::mutex);
|
||||
DiskSpaceMonitor::reserved_bytes -= size;
|
||||
size = new_size;
|
||||
DiskSpaceMonitor::reserved_bytes += size;
|
||||
@ -73,20 +74,21 @@ public:
|
||||
{
|
||||
return size;
|
||||
}
|
||||
private:
|
||||
|
||||
Reservation(size_t size_)
|
||||
: size(size_), metric_increment(CurrentMetrics::DiskSpaceReservedForMerge, size)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(DiskSpaceMonitor::mutex);
|
||||
std::lock_guard<std::mutex> lock(DiskSpaceMonitor::mutex);
|
||||
DiskSpaceMonitor::reserved_bytes += size;
|
||||
++DiskSpaceMonitor::reservation_count;
|
||||
}
|
||||
|
||||
private:
|
||||
size_t size;
|
||||
CurrentMetrics::Increment metric_increment;
|
||||
};
|
||||
|
||||
typedef Poco::SharedPtr<Reservation> ReservationPtr;
|
||||
using ReservationPtr = std::shared_ptr<Reservation>;
|
||||
|
||||
static size_t getUnreservedFreeSpace(const std::string & path)
|
||||
{
|
||||
@ -100,7 +102,7 @@ public:
|
||||
/// Зарезервируем дополнительно 30 МБ. Когда я тестировал, statvfs показывал на несколько мегабайт больше свободного места, чем df.
|
||||
res -= std::min(res, 30 * (1ul << 20));
|
||||
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
if (reserved_bytes > res)
|
||||
res = 0;
|
||||
@ -112,13 +114,13 @@ public:
|
||||
|
||||
static size_t getReservedSpace()
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
return reserved_bytes;
|
||||
}
|
||||
|
||||
static size_t getReservationCount()
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
return reservation_count;
|
||||
}
|
||||
|
||||
@ -129,13 +131,13 @@ public:
|
||||
if (free_bytes < size)
|
||||
throw Exception("Not enough free disk space to reserve: " + formatReadableSizeWithBinarySuffix(free_bytes) + " available, "
|
||||
+ formatReadableSizeWithBinarySuffix(size) + " requested", ErrorCodes::NOT_ENOUGH_SPACE);
|
||||
return new Reservation(size);
|
||||
return std::make_shared<Reservation>(size);
|
||||
}
|
||||
|
||||
private:
|
||||
static size_t reserved_bytes;
|
||||
static size_t reservation_count;
|
||||
static Poco::FastMutex mutex;
|
||||
static std::mutex mutex;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -96,7 +96,7 @@ struct ReplicatedMergeTreeLogEntryData
|
||||
|
||||
struct ReplicatedMergeTreeLogEntry : ReplicatedMergeTreeLogEntryData
|
||||
{
|
||||
typedef Poco::SharedPtr<ReplicatedMergeTreeLogEntry> Ptr;
|
||||
using Ptr = std::shared_ptr<ReplicatedMergeTreeLogEntry>;
|
||||
|
||||
std::condition_variable execution_complete; /// Пробуждается когда currently_executing становится false.
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/Mutex.h>
|
||||
#include <mutex>
|
||||
|
||||
#include <DB/Core/NamesAndTypes.h>
|
||||
#include <DB/Storages/IStorage.h>
|
||||
@ -63,7 +63,7 @@ private:
|
||||
/// Сами данные. list - чтобы при вставке в конец, существующие итераторы не инвалидировались.
|
||||
BlocksList data;
|
||||
|
||||
Poco::FastMutex mutex;
|
||||
std::mutex mutex;
|
||||
|
||||
StorageMemory(
|
||||
const std::string & name_,
|
||||
|
@ -119,7 +119,7 @@ private:
|
||||
SimpleIncrement increment;
|
||||
|
||||
MergeTreeData::DataParts currently_merging;
|
||||
Poco::FastMutex currently_merging_mutex;
|
||||
std::mutex currently_merging_mutex;
|
||||
|
||||
Logger * log;
|
||||
|
||||
@ -152,7 +152,7 @@ private:
|
||||
{
|
||||
try
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(storage.currently_merging_mutex);
|
||||
std::lock_guard<std::mutex> lock(storage.currently_merging_mutex);
|
||||
for (const auto & part : parts)
|
||||
{
|
||||
if (!storage.currently_merging.count(part))
|
||||
@ -167,7 +167,7 @@ private:
|
||||
}
|
||||
};
|
||||
|
||||
typedef Poco::SharedPtr<CurrentlyMergingPartsTagger> CurrentlyMergingPartsTaggerPtr;
|
||||
using CurrentlyMergingPartsTaggerPtr = std::shared_ptr<CurrentlyMergingPartsTagger>;
|
||||
|
||||
StorageMergeTree(
|
||||
const String & path_,
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Parsers/ASTFunction.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
@ -32,7 +30,7 @@ public:
|
||||
virtual ~ITableFunction() {};
|
||||
};
|
||||
|
||||
typedef SharedPtr<ITableFunction> TableFunctionPtr;
|
||||
using TableFunctionPtr = std::shared_ptr<ITableFunction>;
|
||||
|
||||
|
||||
}
|
||||
|
@ -136,7 +136,7 @@ private:
|
||||
Stats info_per_interval;
|
||||
Stats info_total;
|
||||
|
||||
Poco::FastMutex mutex;
|
||||
std::mutex mutex;
|
||||
|
||||
boost::threadpool::pool pool;
|
||||
|
||||
@ -202,7 +202,7 @@ private:
|
||||
{
|
||||
auto total_queries = 0;
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
total_queries = info_total.queries;
|
||||
}
|
||||
printNumberOfQueriesExecuted(total_queries);
|
||||
@ -303,7 +303,7 @@ private:
|
||||
|
||||
double seconds = watch.elapsedSeconds();
|
||||
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
info_per_interval.add(seconds, progress.rows, progress.bytes, info.rows, info.bytes);
|
||||
info_total.add(seconds, progress.rows, progress.bytes, info.rows, info.bytes);
|
||||
}
|
||||
@ -311,7 +311,7 @@ private:
|
||||
|
||||
void report(Stats & info)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
double seconds = info.watch.elapsedSeconds();
|
||||
|
||||
|
@ -139,7 +139,7 @@ private:
|
||||
|
||||
winsize terminal_size {}; /// Размер терминала - для вывода прогресс-бара.
|
||||
|
||||
SharedPtr<Connection> connection; /// Соединение с БД.
|
||||
std::unique_ptr<Connection> connection; /// Соединение с БД.
|
||||
String query; /// Текущий запрос.
|
||||
|
||||
String format; /// Формат вывода результата в консоль.
|
||||
@ -394,7 +394,7 @@ private:
|
||||
<< (!user.empty() ? " as user " + user : "")
|
||||
<< "." << std::endl;
|
||||
|
||||
connection = new Connection(host, port, default_database, user, password, "client", compression,
|
||||
connection = std::make_unique<Connection>(host, port, default_database, user, password, "client", compression,
|
||||
Poco::Timespan(config().getInt("connect_timeout", DBMS_DEFAULT_CONNECT_TIMEOUT_SEC), 0),
|
||||
Poco::Timespan(config().getInt("receive_timeout", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0),
|
||||
Poco::Timespan(config().getInt("send_timeout", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0));
|
||||
|
@ -63,7 +63,7 @@ MultiplexedConnections::MultiplexedConnections(ConnectionPools & pools_, const S
|
||||
|
||||
for (auto & pool : pools_)
|
||||
{
|
||||
if (pool.isNull())
|
||||
if (pool)
|
||||
throw Exception("Invalid pool specified", ErrorCodes::LOGICAL_ERROR);
|
||||
initFromShard(pool.get());
|
||||
}
|
||||
@ -78,7 +78,7 @@ MultiplexedConnections::MultiplexedConnections(ConnectionPools & pools_, const S
|
||||
|
||||
void MultiplexedConnections::sendExternalTablesData(std::vector<ExternalTablesData> & data)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(cancel_mutex);
|
||||
std::lock_guard<std::mutex> lock(cancel_mutex);
|
||||
|
||||
if (!sent_query)
|
||||
throw Exception("Cannot send external tables data: query not yet sent.", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -99,7 +99,7 @@ void MultiplexedConnections::sendExternalTablesData(std::vector<ExternalTablesDa
|
||||
|
||||
void MultiplexedConnections::sendQuery(const String & query, const String & query_id, UInt64 stage, bool with_pending_data)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(cancel_mutex);
|
||||
std::lock_guard<std::mutex> lock(cancel_mutex);
|
||||
|
||||
if (sent_query)
|
||||
throw Exception("Query already sent.", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -159,7 +159,7 @@ void MultiplexedConnections::sendQuery(const String & query, const String & quer
|
||||
|
||||
Connection::Packet MultiplexedConnections::receivePacket()
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(cancel_mutex);
|
||||
std::lock_guard<std::mutex> lock(cancel_mutex);
|
||||
const auto & packet = receivePacketUnlocked();
|
||||
if (block_extra_info)
|
||||
{
|
||||
@ -181,7 +181,7 @@ BlockExtraInfo MultiplexedConnections::getBlockExtraInfo() const
|
||||
|
||||
void MultiplexedConnections::disconnect()
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(cancel_mutex);
|
||||
std::lock_guard<std::mutex> lock(cancel_mutex);
|
||||
|
||||
for (auto it = replica_map.begin(); it != replica_map.end(); ++it)
|
||||
{
|
||||
@ -197,7 +197,7 @@ void MultiplexedConnections::disconnect()
|
||||
|
||||
void MultiplexedConnections::sendCancel()
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(cancel_mutex);
|
||||
std::lock_guard<std::mutex> lock(cancel_mutex);
|
||||
|
||||
if (!sent_query || cancelled)
|
||||
throw Exception("Cannot cancel. Either no query sent or already cancelled.", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -215,7 +215,7 @@ void MultiplexedConnections::sendCancel()
|
||||
|
||||
Connection::Packet MultiplexedConnections::drain()
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(cancel_mutex);
|
||||
std::lock_guard<std::mutex> lock(cancel_mutex);
|
||||
|
||||
if (!cancelled)
|
||||
throw Exception("Cannot drain connections: cancel first.", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -250,7 +250,7 @@ Connection::Packet MultiplexedConnections::drain()
|
||||
|
||||
std::string MultiplexedConnections::dumpAddresses() const
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(cancel_mutex);
|
||||
std::lock_guard<std::mutex> lock(cancel_mutex);
|
||||
return dumpAddressesUnlocked();
|
||||
}
|
||||
|
||||
|
@ -50,7 +50,7 @@ struct __attribute__((__aligned__(64))) AlignedSmallLock : public SmallLock
|
||||
};
|
||||
|
||||
|
||||
typedef Poco::FastMutex Mutex;
|
||||
typedef std::mutex Mutex;
|
||||
|
||||
|
||||
/*typedef HashTableWithSmallLocks<
|
||||
|
@ -255,7 +255,7 @@ struct Work
|
||||
|
||||
typedef HashMap<Key, Value, HashCRC32<Key>> Map;
|
||||
typedef TwoLevelHashMap<Key, Value, HashCRC32<Key>> MapTwoLevel;
|
||||
typedef Poco::FastMutex Mutex;
|
||||
typedef std::mutex Mutex;
|
||||
|
||||
|
||||
struct Creator
|
||||
|
@ -1,7 +1,7 @@
|
||||
#include <iomanip>
|
||||
#include <random>
|
||||
|
||||
/*#include <Poco/Mutex.h>
|
||||
/*#include <mutex>
|
||||
#include <Poco/Ext/ThreadNumber.h>*/
|
||||
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
@ -50,8 +50,8 @@ Block IProfilingBlockInputStream::read()
|
||||
|
||||
/* if (res)
|
||||
{
|
||||
static Poco::FastMutex mutex;
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
static std::mutex mutex;
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
std::cerr << std::endl;
|
||||
std::cerr << "[ " << Poco::ThreadNumber::get() << " ]\t" << getName() << std::endl;
|
||||
|
@ -240,7 +240,7 @@ void RemoteBlockInputStream::createMultiplexedConnections()
|
||||
else if (pool != nullptr)
|
||||
multiplexed_connections = std::make_unique<MultiplexedConnections>(pool, multiplexed_connections_settings, throttler,
|
||||
append_extra_info, pool_mode);
|
||||
else if (!pools.isNull())
|
||||
else if (pools != nullptr)
|
||||
multiplexed_connections = std::make_unique<MultiplexedConnections>(*pools, multiplexed_connections_settings, throttler,
|
||||
append_extra_info, pool_mode);
|
||||
else
|
||||
@ -260,14 +260,7 @@ void RemoteBlockInputStream::init(const Settings * settings_)
|
||||
|
||||
void RemoteBlockInputStream::sendQuery()
|
||||
{
|
||||
try
|
||||
{
|
||||
createMultiplexedConnections();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
throw;
|
||||
}
|
||||
createMultiplexedConnections();
|
||||
|
||||
if (settings.skip_unavailable_shards && 0 == multiplexed_connections->size())
|
||||
return;
|
||||
|
@ -160,7 +160,7 @@ void TotalsHavingBlockInputStream::addToTotals(Block & totals, Block & block, co
|
||||
|
||||
ArenaPtr arena;
|
||||
if (init)
|
||||
arena = new Arena;
|
||||
arena = std::make_shared<Arena>();
|
||||
|
||||
for (size_t i = 0; i < block.columns(); ++i)
|
||||
{
|
||||
|
@ -29,7 +29,7 @@ static void glue(BlockInputStreamPtr & node, IDsMap & ids_map, ForksMap & forks_
|
||||
/// Вставить "вилку" или использовать уже готовую.
|
||||
if (forks_map.end() == forks_map.find(id))
|
||||
{
|
||||
forks_map[id] = new ForkBlockInputStreams(node);
|
||||
forks_map[id] = std::make_shared<ForkBlockInputStreams>(node);
|
||||
std::cerr << "Forking at " << id << std::endl;
|
||||
}
|
||||
|
||||
|
@ -15,11 +15,11 @@ using Poco::SharedPtr;
|
||||
using namespace DB;
|
||||
|
||||
|
||||
void inputThread(BlockInputStreamPtr in, BlockOutputStreamPtr out, WriteBuffer & wb, Poco::FastMutex & mutex)
|
||||
void inputThread(BlockInputStreamPtr in, BlockOutputStreamPtr out, WriteBuffer & wb, std::mutex & mutex)
|
||||
{
|
||||
while (Block block = in->read())
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
out->write(block);
|
||||
wb.next();
|
||||
@ -79,7 +79,7 @@ int main(int argc, char ** argv)
|
||||
|
||||
std::cerr << forks.size() << std::endl;
|
||||
|
||||
Poco::FastMutex mutex;
|
||||
std::mutex mutex;
|
||||
|
||||
boost::threadpool::pool pool(inputs.size() + forks.size());
|
||||
|
||||
|
@ -812,7 +812,7 @@ void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants, si
|
||||
|
||||
/// NOTE Вместо освобождения памяти и создания новых хэш-таблиц и арены, можно переиспользовать старые.
|
||||
data_variants.init(data_variants.type);
|
||||
data_variants.aggregates_pools = Arenas(1, new Arena);
|
||||
data_variants.aggregates_pools = Arenas(1, std::make_shared<Arena>());
|
||||
data_variants.aggregates_pool = data_variants.aggregates_pools.back().get();
|
||||
|
||||
block_out.flush();
|
||||
@ -1987,7 +1987,7 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
|
||||
if (bucket == -1)
|
||||
continue;
|
||||
|
||||
result.aggregates_pools.push_back(new Arena);
|
||||
result.aggregates_pools.push_back(std::make_shared<Arena>());
|
||||
Arena * aggregates_pool = result.aggregates_pools.back().get();
|
||||
|
||||
tasks[bucket] = std::packaged_task<void()>(std::bind(merge_bucket, bucket, aggregates_pool, current_memory_tracker));
|
||||
|
@ -161,7 +161,7 @@ Cluster::Cluster(const Settings & settings, const String & cluster_name)
|
||||
else
|
||||
{
|
||||
info.dir_names.push_back(addressToDirName(address));
|
||||
info.pool = new ConnectionPool(
|
||||
info.pool = std::make_shared<ConnectionPool>(
|
||||
settings.distributed_connections_pool_size,
|
||||
address.host_name, address.port, address.resolved_address,
|
||||
"", address.user, address.password,
|
||||
@ -240,7 +240,7 @@ Cluster::Cluster(const Settings & settings, const String & cluster_name)
|
||||
shard_local_addresses.push_back(replica);
|
||||
else
|
||||
{
|
||||
replicas.emplace_back(new ConnectionPool(
|
||||
replicas.emplace_back(std::make_shared<ConnectionPool>(
|
||||
settings.distributed_connections_pool_size,
|
||||
replica.host_name, replica.port, replica.resolved_address,
|
||||
"", replica.user, replica.password,
|
||||
@ -253,7 +253,7 @@ Cluster::Cluster(const Settings & settings, const String & cluster_name)
|
||||
|
||||
ConnectionPoolPtr shard_pool;
|
||||
if (!replicas.empty())
|
||||
shard_pool = new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries);
|
||||
shard_pool = std::make_shared<ConnectionPoolWithFailover>(replicas, settings.load_balancing, settings.connections_with_failover_max_tries);
|
||||
|
||||
slot_to_shard.insert(std::end(slot_to_shard), weight, shards_info.size());
|
||||
shards_info.push_back({std::move(dir_names), current_shard_num, weight, shard_local_addresses, shard_pool});
|
||||
@ -289,7 +289,7 @@ Cluster::Cluster(const Settings & settings, std::vector<std::vector<String>> nam
|
||||
|
||||
for (const auto & replica : current)
|
||||
{
|
||||
replicas.emplace_back(new ConnectionPool(
|
||||
replicas.emplace_back(std::make_shared<ConnectionPool>(
|
||||
settings.distributed_connections_pool_size,
|
||||
replica.host_name, replica.port, replica.resolved_address,
|
||||
"", replica.user, replica.password,
|
||||
@ -299,7 +299,8 @@ Cluster::Cluster(const Settings & settings, std::vector<std::vector<String>> nam
|
||||
saturate(settings.send_timeout, settings.limits.max_execution_time)));
|
||||
}
|
||||
|
||||
ConnectionPoolPtr shard_pool = new ConnectionPoolWithFailover(replicas, settings.load_balancing, settings.connections_with_failover_max_tries);
|
||||
ConnectionPoolPtr shard_pool = std::make_shared<ConnectionPoolWithFailover>(
|
||||
replicas, settings.load_balancing, settings.connections_with_failover_max_tries);
|
||||
|
||||
slot_to_shard.insert(std::end(slot_to_shard), default_weight, shards_info.size());
|
||||
shards_info.push_back({{}, current_shard_num, default_weight, {}, shard_pool});
|
||||
|
@ -105,14 +105,14 @@ BlockInputStreams Query::execute()
|
||||
|
||||
if (actual_pools_per_thread == 1)
|
||||
{
|
||||
res.emplace_back(query_constructor.createRemote(shard_info.pool, query, new_settings, throttler, context));
|
||||
res.emplace_back(query_constructor.createRemote(shard_info.pool.get(), query, new_settings, throttler, context));
|
||||
++current_thread;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (do_init)
|
||||
{
|
||||
pools = new ConnectionPools;
|
||||
pools = std::shared_ptr<ConnectionPools>();
|
||||
do_init = false;
|
||||
}
|
||||
|
||||
|
@ -841,7 +841,7 @@ BackgroundProcessingPool & Context::getBackgroundPool()
|
||||
{
|
||||
auto lock = getLock();
|
||||
if (!shared->background_pool)
|
||||
shared->background_pool = new BackgroundProcessingPool(settings.background_pool_size);
|
||||
shared->background_pool = std::make_shared<BackgroundProcessingPool>(settings.background_pool_size);
|
||||
return *shared->background_pool;
|
||||
}
|
||||
|
||||
|
@ -287,7 +287,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns)
|
||||
|
||||
column_declaration->name = column.name;
|
||||
|
||||
StringPtr type_name{new String(column.type->getName())};
|
||||
StringPtr type_name = std::make_shared<String>(column.type->getName());
|
||||
auto pos = type_name->data();
|
||||
const auto end = pos + type_name->size();
|
||||
|
||||
@ -318,7 +318,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(NamesAndTypesList columns,
|
||||
|
||||
column_declaration->name = column.name;
|
||||
|
||||
StringPtr type_name{new String(column.type->getName())};
|
||||
StringPtr type_name = std::make_shared<String>(column.type->getName());
|
||||
auto pos = type_name->data();
|
||||
const auto end = pos + type_name->size();
|
||||
|
||||
|
@ -17,7 +17,7 @@ ProcessList::EntryPtr ProcessList::insert(
|
||||
EntryPtr res;
|
||||
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
if (max_size && cur_size >= max_size
|
||||
&& (!settings.queue_max_wait_ms.totalMilliseconds() || !have_space.tryWait(mutex, settings.queue_max_wait_ms.totalMilliseconds())))
|
||||
@ -84,7 +84,7 @@ ProcessList::EntryPtr ProcessList::insert(
|
||||
|
||||
ProcessListEntry::~ProcessListEntry()
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(parent.mutex);
|
||||
std::lock_guard<std::mutex> lock(parent.mutex);
|
||||
|
||||
/// Важен порядок удаления memory_tracker-ов.
|
||||
|
||||
@ -131,7 +131,7 @@ ProcessListEntry::~ProcessListEntry()
|
||||
|
||||
void ProcessList::addTemporaryTable(ProcessListElement & elem, const String & table_name, StoragePtr storage)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
elem.temporary_tables[table_name] = storage;
|
||||
}
|
||||
@ -139,7 +139,7 @@ void ProcessList::addTemporaryTable(ProcessListElement & elem, const String & ta
|
||||
|
||||
StoragePtr ProcessList::tryGetTemporaryTable(const String & query_id, const String & table_name) const
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
/// NOTE Ищем по всем user-ам. То есть, нет изоляции, и сложность O(users).
|
||||
for (const auto & user_queries : user_to_queries)
|
||||
|
@ -269,7 +269,7 @@ QuotaForIntervalsPtr Quota::get(const String & quota_key, const String & user_na
|
||||
? quota_key
|
||||
: user_name));
|
||||
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
Container::iterator it = quota_for_keys.find(quota_key_hashed);
|
||||
if (quota_for_keys.end() == it)
|
||||
|
@ -21,7 +21,7 @@ ActiveDataPartSet::ActiveDataPartSet(const Strings & names)
|
||||
|
||||
void ActiveDataPartSet::add(const String & name)
|
||||
{
|
||||
Poco::ScopedLock<Poco::Mutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
addImpl(name);
|
||||
}
|
||||
|
||||
@ -62,7 +62,7 @@ void ActiveDataPartSet::addImpl(const String & name)
|
||||
|
||||
String ActiveDataPartSet::getContainingPart(const String & part_name) const
|
||||
{
|
||||
Poco::ScopedLock<Poco::Mutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
Part part;
|
||||
parsePartName(part_name, part);
|
||||
@ -91,7 +91,7 @@ String ActiveDataPartSet::getContainingPart(const String & part_name) const
|
||||
|
||||
Strings ActiveDataPartSet::getParts() const
|
||||
{
|
||||
Poco::ScopedLock<Poco::Mutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
Strings res;
|
||||
res.reserve(parts.size());
|
||||
@ -104,7 +104,7 @@ Strings ActiveDataPartSet::getParts() const
|
||||
|
||||
size_t ActiveDataPartSet::size() const
|
||||
{
|
||||
Poco::ScopedLock<Poco::Mutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
return parts.size();
|
||||
}
|
||||
|
||||
|
@ -5,6 +5,6 @@ namespace DB
|
||||
|
||||
size_t DiskSpaceMonitor::reserved_bytes;
|
||||
size_t DiskSpaceMonitor::reservation_count;
|
||||
Poco::FastMutex DiskSpaceMonitor::mutex;
|
||||
std::mutex DiskSpaceMonitor::mutex;
|
||||
|
||||
}
|
||||
|
@ -140,7 +140,7 @@ String ReplicatedMergeTreeLogEntryData::toString() const
|
||||
ReplicatedMergeTreeLogEntry::Ptr ReplicatedMergeTreeLogEntry::parse(const String & s, const zkutil::Stat & stat)
|
||||
{
|
||||
ReadBufferFromString in(s);
|
||||
Ptr res = new ReplicatedMergeTreeLogEntry;
|
||||
Ptr res = std::make_shared<ReplicatedMergeTreeLogEntry>();
|
||||
res->readText(in);
|
||||
assertEOF(in);
|
||||
|
||||
|
@ -69,7 +69,7 @@ public:
|
||||
void write(const Block & block)
|
||||
{
|
||||
storage.check(block, true);
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(storage.mutex);
|
||||
std::lock_guard<std::mutex> lock(storage.mutex);
|
||||
storage.data.push_back(block);
|
||||
}
|
||||
private:
|
||||
@ -132,7 +132,7 @@ BlockInputStreams StorageMemory::read(
|
||||
check(column_names);
|
||||
processed_stage = QueryProcessingStage::FetchColumns;
|
||||
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
size_t size = data.size();
|
||||
|
||||
@ -165,7 +165,7 @@ BlockOutputStreamPtr StorageMemory::write(
|
||||
|
||||
void StorageMemory::drop()
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
data.clear();
|
||||
}
|
||||
|
||||
|
@ -270,7 +270,7 @@ bool StorageMergeTree::merge(
|
||||
String merged_name;
|
||||
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(currently_merging_mutex);
|
||||
std::lock_guard<std::mutex> lock(currently_merging_mutex);
|
||||
|
||||
MergeTreeData::DataPartsVector parts;
|
||||
auto can_merge = std::bind(&StorageMergeTree::canMergeParts, this, std::placeholders::_1, std::placeholders::_2);
|
||||
@ -295,7 +295,8 @@ bool StorageMergeTree::merge(
|
||||
if (!selected)
|
||||
return false;
|
||||
|
||||
merging_tagger = new CurrentlyMergingPartsTagger(parts, MergeTreeDataMerger::estimateDiskSpaceForMerge(parts), *this);
|
||||
merging_tagger = std::make_shared<CurrentlyMergingPartsTagger>(
|
||||
parts, MergeTreeDataMerger::estimateDiskSpaceForMerge(parts), *this);
|
||||
|
||||
/// Если собираемся сливать большие куски, увеличим счетчик потоков, сливающих большие куски.
|
||||
if (pool_context)
|
||||
|
@ -346,7 +346,7 @@ StoragePtr StorageReplicatedMergeTree::create(
|
||||
|
||||
auto get_endpoint_holder = [&res](InterserverIOEndpointPtr endpoint)
|
||||
{
|
||||
return new InterserverIOEndpointHolder(
|
||||
return std::make_shared<InterserverIOEndpointHolder>(
|
||||
endpoint->getId(res->replica_path),
|
||||
endpoint,
|
||||
res->context.getInterserverIOHandler());
|
||||
@ -355,29 +355,29 @@ StoragePtr StorageReplicatedMergeTree::create(
|
||||
if (res->tryGetZooKeeper())
|
||||
{
|
||||
{
|
||||
InterserverIOEndpointPtr endpoint = new DataPartsExchange::Service(res->data, res_ptr);
|
||||
InterserverIOEndpointPtr endpoint = std::make_shared<DataPartsExchange::Service>(res->data, res_ptr);
|
||||
res->endpoint_holder = get_endpoint_holder(endpoint);
|
||||
}
|
||||
|
||||
/// Сервисы для перешардирования.
|
||||
|
||||
{
|
||||
InterserverIOEndpointPtr endpoint = new RemoteDiskSpaceMonitor::Service(res->context);
|
||||
InterserverIOEndpointPtr endpoint = std::make_shared<RemoteDiskSpaceMonitor::Service>(res->context);
|
||||
res->disk_space_monitor_endpoint_holder = get_endpoint_holder(endpoint);
|
||||
}
|
||||
|
||||
{
|
||||
InterserverIOEndpointPtr endpoint = new ShardedPartitionUploader::Service(res_ptr);
|
||||
InterserverIOEndpointPtr endpoint = std::make_shared<ShardedPartitionUploader::Service>(res_ptr);
|
||||
res->sharded_partition_uploader_endpoint_holder = get_endpoint_holder(endpoint);
|
||||
}
|
||||
|
||||
{
|
||||
InterserverIOEndpointPtr endpoint = new RemoteQueryExecutor::Service(res->context);
|
||||
InterserverIOEndpointPtr endpoint = std::make_shared<RemoteQueryExecutor::Service>(res->context);
|
||||
res->remote_query_executor_endpoint_holder = get_endpoint_holder(endpoint);
|
||||
}
|
||||
|
||||
{
|
||||
InterserverIOEndpointPtr endpoint = new RemotePartChecker::Service(res_ptr);
|
||||
InterserverIOEndpointPtr endpoint = std::make_shared<RemotePartChecker::Service>(res_ptr);
|
||||
res->remote_part_checker_endpoint_holder = get_endpoint_holder(endpoint);
|
||||
}
|
||||
}
|
||||
@ -1124,7 +1124,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry, Backgro
|
||||
size_t aio_threshold = context.getSettings().min_bytes_to_use_direct_io;
|
||||
|
||||
auto part = merger.mergePartsToTemporaryPart(
|
||||
parts, entry.new_part_name, *merge_entry, aio_threshold, entry.create_time, reserved_space);
|
||||
parts, entry.new_part_name, *merge_entry, aio_threshold, entry.create_time, reserved_space.get());
|
||||
|
||||
zkutil::Ops ops;
|
||||
|
||||
@ -1817,7 +1817,7 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n
|
||||
|
||||
String part_path = replica_path + "/parts/" + part_name;
|
||||
|
||||
LogEntryPtr log_entry = new LogEntry;
|
||||
LogEntryPtr log_entry = std::make_shared<LogEntry>();
|
||||
log_entry->type = LogEntry::GET_PART;
|
||||
log_entry->create_time = tryGetPartCreateTime(zookeeper, replica_path, part_name);
|
||||
log_entry->source_replica = "";
|
||||
|
@ -22,9 +22,9 @@ TableFunctionPtr TableFunctionFactory::get(
|
||||
if (context.getSettings().limits.readonly == 1) /** Например, для readonly = 2 - разрешено. */
|
||||
throw Exception("Table functions are forbidden in readonly mode", ErrorCodes::READONLY);
|
||||
|
||||
if (name == "merge") return new TableFunctionMerge;
|
||||
else if (name == "remote") return new TableFunctionRemote;
|
||||
else if (name == "shardByHash") return new TableFunctionShardByHash;
|
||||
if (name == "merge") return std::make_shared<TableFunctionMerge>();
|
||||
else if (name == "remote") return std::make_shared<TableFunctionRemote>();
|
||||
else if (name == "shardByHash") return std::make_shared<TableFunctionShardByHash>();
|
||||
else
|
||||
throw Exception("Unknown table function " + name, ErrorCodes::UNKNOWN_FUNCTION);
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user