Merge branch 'master' of github.com:yandex/ClickHouse

This commit is contained in:
Alexey Milovidov 2016-05-30 21:24:35 +03:00
commit a4a543f1c3
505 changed files with 4790 additions and 5131 deletions

View File

@ -8,7 +8,7 @@ add_definitions(
include_directories (include src)
add_library (tcmalloc_minimal
add_library (tcmalloc_minimal_internal
./src/malloc_hook.cc
./src/base/spinlock_internal.cc
./src/base/spinlock.cc

View File

@ -753,6 +753,10 @@ add_library (dbms
src/DataTypes/DataTypeTuple.cpp
src/DataTypes/FieldToDataType.cpp
src/Dictionaries/MySQLDictionarySource.cpp
src/Dictionaries/ODBCDictionarySource.cpp
src/Dictionaries/writeParenthesisedString.cpp
src/Parsers/ASTSelectQuery.cpp
src/Parsers/ASTAlterQuery.cpp
src/Parsers/ASTLiteral.cpp
@ -935,7 +939,6 @@ target_link_libraries(dbms
libcrypto.a
libboost_thread.a
libboost_system.a
librt.a
${LINK_MONGOCLIENT}
libboost_regex.a
PocoData

View File

@ -31,7 +31,7 @@ public:
DataTypePtr getReturnType() const override
{
return new DataTypeFloat64;
return std::make_shared<DataTypeFloat64>();
}
void setArgument(const DataTypePtr & argument)

View File

@ -27,7 +27,7 @@ public:
DataTypePtr getReturnType() const override
{
return new DataTypeUInt64;
return std::make_shared<DataTypeUInt64>();
}

View File

@ -44,7 +44,7 @@ public:
DataTypePtr getReturnType() const override
{
return new DataTypeArray(new typename DataTypeFromFieldType<T>::Type);
return std::make_shared<DataTypeArray>(std::make_shared<typename DataTypeFromFieldType<T>::Type>());
}
void setArgument(const DataTypePtr & argument)
@ -119,7 +119,7 @@ public:
DataTypePtr getReturnType() const override
{
return new DataTypeArray(type);
return std::make_shared<DataTypeArray>(type);
}
void setArgument(const DataTypePtr & argument)

View File

@ -39,14 +39,14 @@ template <typename T>
class AggregateFunctionGroupUniqArray : public IUnaryAggregateFunction<AggregateFunctionGroupUniqArrayData<T>, AggregateFunctionGroupUniqArray<T> >
{
private:
typedef AggregateFunctionGroupUniqArrayData<T> State;
using State = AggregateFunctionGroupUniqArrayData<T>;
public:
String getName() const override { return "groupUniqArray"; }
DataTypePtr getReturnType() const override
{
return new DataTypeArray(new typename DataTypeFromFieldType<T>::Type);
return std::make_shared<DataTypeArray>(std::make_shared<typename DataTypeFromFieldType<T>::Type>());
}
void setArgument(const DataTypePtr & argument)

View File

@ -21,7 +21,7 @@ namespace DB
template <typename ArgumentFieldType>
struct AggregateFunctionQuantileData
{
typedef ReservoirSampler<ArgumentFieldType, ReservoirSamplerOnEmpty::RETURN_NAN_OR_ZERO> Sample;
using Sample = ReservoirSampler<ArgumentFieldType, ReservoirSamplerOnEmpty::RETURN_NAN_OR_ZERO>;
Sample sample; /// TODO Добавить MemoryTracker
};
@ -54,7 +54,7 @@ public:
void setArgument(const DataTypePtr & argument)
{
if (returns_float)
type = new DataTypeFloat64;
type = std::make_shared<DataTypeFloat64>();
else
type = argument;
}
@ -121,13 +121,13 @@ public:
DataTypePtr getReturnType() const override
{
return new DataTypeArray(type);
return std::make_shared<DataTypeArray>(type);
}
void setArgument(const DataTypePtr & argument)
{
if (returns_float)
type = new DataTypeFloat64;
type = std::make_shared<DataTypeFloat64>();
else
type = argument;
}

View File

@ -55,7 +55,7 @@ public:
void setArgumentsImpl(const DataTypes & arguments)
{
type = returns_float ? new DataTypeFloat64 : arguments[0];
type = returns_float ? std::make_shared<DataTypeFloat64>() : arguments[0];
if (!arguments[1]->isNumeric())
throw Exception{
@ -130,12 +130,12 @@ public:
DataTypePtr getReturnType() const override
{
return new DataTypeArray(type);
return std::make_shared<DataTypeArray>(type);
}
void setArgumentsImpl(const DataTypes & arguments)
{
type = returns_float ? new DataTypeFloat64 : arguments[0];
type = returns_float ? std::make_shared<DataTypeFloat64>() : arguments[0];
if (!arguments[1]->isNumeric())
throw Exception{

View File

@ -137,7 +137,7 @@ public:
DataTypePtr getReturnType() const override
{
return new DataTypeArray(type);
return std::make_shared<DataTypeArray>(type);
}
void setArgument(const DataTypePtr & argument)

View File

@ -176,7 +176,7 @@ public:
DataTypePtr getReturnType() const override
{
return new DataTypeArray(type);
return std::make_shared<DataTypeArray>(type);
}
void setArgumentsImpl(const DataTypes & arguments)

View File

@ -369,7 +369,7 @@ public:
void setArgument(const DataTypePtr & argument)
{
if (returns_float)
type = new DataTypeFloat32;
type = std::make_shared<DataTypeFloat32>();
else
type = argument;
}
@ -436,7 +436,7 @@ public:
void setArgumentsImpl(const DataTypes & arguments)
{
if (returns_float)
type = new DataTypeFloat32;
type = std::make_shared<DataTypeFloat32>();
else
type = arguments.at(0);
}
@ -497,13 +497,13 @@ public:
DataTypePtr getReturnType() const override
{
return new DataTypeArray(type);
return std::make_shared<DataTypeArray>(type);
}
void setArgument(const DataTypePtr & argument)
{
if (returns_float)
type = new DataTypeFloat32;
type = std::make_shared<DataTypeFloat32>();
else
type = argument;
}
@ -577,13 +577,13 @@ public:
DataTypePtr getReturnType() const override
{
return new DataTypeArray(type);
return std::make_shared<DataTypeArray>(type);
}
void setArgumentsImpl(const DataTypes & arguments)
{
if (returns_float)
type = new DataTypeFloat32;
type = std::make_shared<DataTypeFloat32>();
else
type = arguments.at(0);
}

View File

@ -798,7 +798,7 @@ public:
DataTypePtr getReturnType() const override
{
return new DataTypeFloat32;
return std::make_shared<DataTypeFloat32>();
}
void setArgument(const DataTypePtr & argument)
@ -857,7 +857,7 @@ public:
DataTypePtr getReturnType() const override
{
return new DataTypeFloat32;
return std::make_shared<DataTypeFloat32>();
}
void setArgumentsImpl(const DataTypes & arguments)
@ -916,7 +916,7 @@ public:
DataTypePtr getReturnType() const override
{
return new DataTypeArray(new DataTypeFloat32);
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeFloat32>());
}
void setArgument(const DataTypePtr & argument)
@ -978,7 +978,7 @@ public:
DataTypePtr getReturnType() const override
{
return new DataTypeArray(new DataTypeFloat32);
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeFloat32>());
}
void setArgumentsImpl(const DataTypes & arguments)

View File

@ -140,7 +140,7 @@ public:
String getName() const override { return "sequenceMatch"; }
DataTypePtr getReturnType() const override { return new DataTypeUInt8; }
DataTypePtr getReturnType() const override { return std::make_shared<DataTypeUInt8>(); }
void setParameters(const Array & params) override
{
@ -504,7 +504,7 @@ class AggregateFunctionSequenceCount final : public AggregateFunctionSequenceMat
public:
String getName() const override { return "sequenceCount"; }
DataTypePtr getReturnType() const override { return new DataTypeUInt64; }
DataTypePtr getReturnType() const override { return std::make_shared<DataTypeUInt64>(); }
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{

View File

@ -32,7 +32,7 @@ public:
DataTypePtr getReturnType() const override
{
return new DataTypeAggregateFunction(nested_func_owner, arguments, params);
return std::make_shared<DataTypeAggregateFunction>(nested_func_owner, arguments, params);
}
void setArguments(const DataTypes & arguments_) override

View File

@ -29,7 +29,7 @@ public:
DataTypePtr getReturnType() const override
{
return new typename DataTypeFromFieldType<typename NearestFieldType<T>::Type>::Type;
return std::make_shared<typename DataTypeFromFieldType<typename NearestFieldType<T>::Type>::Type>();
}
void setArgument(const DataTypePtr & argument)

View File

@ -33,7 +33,7 @@ namespace DB
struct AggregateFunctionUniqUniquesHashSetData
{
typedef UniquesHashSet<DefaultHash<UInt64>> Set;
using Set = UniquesHashSet<DefaultHash<UInt64>>;
Set set;
static String getName() { return "uniq"; }
@ -42,7 +42,7 @@ struct AggregateFunctionUniqUniquesHashSetData
/// Для функции, принимающей несколько аргументов. Такая функция сама заранее их хэширует, поэтому здесь используется TrivialHash.
struct AggregateFunctionUniqUniquesHashSetDataForVariadic
{
typedef UniquesHashSet<TrivialHash> Set;
using Set = UniquesHashSet<TrivialHash>;
Set set;
static String getName() { return "uniq"; }
@ -54,7 +54,7 @@ struct AggregateFunctionUniqUniquesHashSetDataForVariadic
template <typename T>
struct AggregateFunctionUniqHLL12Data
{
typedef HyperLogLogWithSmallSetOptimization<T, 16, 12> Set;
using Set = HyperLogLogWithSmallSetOptimization<T, 16, 12>;
Set set;
static String getName() { return "uniqHLL12"; }
@ -63,7 +63,7 @@ struct AggregateFunctionUniqHLL12Data
template <>
struct AggregateFunctionUniqHLL12Data<String>
{
typedef HyperLogLogWithSmallSetOptimization<UInt64, 16, 12> Set;
using Set = HyperLogLogWithSmallSetOptimization<UInt64, 16, 12>;
Set set;
static String getName() { return "uniqHLL12"; }
@ -71,7 +71,7 @@ struct AggregateFunctionUniqHLL12Data<String>
struct AggregateFunctionUniqHLL12DataForVariadic
{
typedef HyperLogLogWithSmallSetOptimization<UInt64, 16, 12, TrivialHash> Set;
using Set = HyperLogLogWithSmallSetOptimization<UInt64, 16, 12, TrivialHash>;
Set set;
static String getName() { return "uniqHLL12"; }
@ -83,7 +83,7 @@ struct AggregateFunctionUniqHLL12DataForVariadic
template <typename T>
struct AggregateFunctionUniqExactData
{
typedef T Key;
using Key = T;
/// При создании, хэш-таблица должна быть небольшой.
typedef HashSet<
@ -102,7 +102,7 @@ struct AggregateFunctionUniqExactData
template <>
struct AggregateFunctionUniqExactData<String>
{
typedef UInt128 Key;
using Key = UInt128;
/// При создании, хэш-таблица должна быть небольшой.
typedef HashSet<
@ -334,7 +334,7 @@ public:
DataTypePtr getReturnType() const override
{
return new DataTypeUInt64;
return std::make_shared<DataTypeUInt64>();
}
void setArgument(const DataTypePtr & argument)
@ -385,7 +385,7 @@ public:
DataTypePtr getReturnType() const override
{
return new DataTypeUInt64;
return std::make_shared<DataTypeUInt64>();
}
void setArguments(const DataTypes & arguments) override

View File

@ -130,7 +130,7 @@ public:
DataTypePtr getReturnType() const override
{
return new DataTypeUInt64;
return std::make_shared<DataTypeUInt64>();
}
void setArgument(const DataTypePtr & argument)
@ -199,7 +199,7 @@ public:
DataTypePtr getReturnType() const override
{
return new DataTypeUInt64;
return std::make_shared<DataTypeUInt64>();
}
void setArguments(const DataTypes & arguments) override

View File

@ -22,7 +22,7 @@ namespace DB
template <typename T>
struct SingleValueDataFixed
{
typedef SingleValueDataFixed<T> Self;
using Self = SingleValueDataFixed<T>;
bool has_value = false; /// Надо запомнить, было ли передано хотя бы одно значение. Это нужно для AggregateFunctionIf.
T value;
@ -141,7 +141,7 @@ struct SingleValueDataFixed
*/
struct __attribute__((__packed__, __aligned__(1))) SingleValueDataString
{
typedef SingleValueDataString Self;
using Self = SingleValueDataString;
Int32 size = -1; /// -1 обозначает, что значения нет.
@ -344,7 +344,7 @@ static_assert(
/// Для любых других типов значений.
struct SingleValueDataGeneric
{
typedef SingleValueDataGeneric Self;
using Self = SingleValueDataGeneric;
Field value;
@ -487,7 +487,7 @@ struct SingleValueDataGeneric
template <typename Data>
struct AggregateFunctionMinData : Data
{
typedef AggregateFunctionMinData<Data> Self;
using Self = AggregateFunctionMinData<Data>;
bool changeIfBetter(const IColumn & column, size_t row_num) { return this->changeIfLess(column, row_num); }
bool changeIfBetter(const Self & to) { return this->changeIfLess(to); }
@ -498,7 +498,7 @@ struct AggregateFunctionMinData : Data
template <typename Data>
struct AggregateFunctionMaxData : Data
{
typedef AggregateFunctionMaxData<Data> Self;
using Self = AggregateFunctionMaxData<Data>;
bool changeIfBetter(const IColumn & column, size_t row_num) { return this->changeIfGreater(column, row_num); }
bool changeIfBetter(const Self & to) { return this->changeIfGreater(to); }
@ -509,7 +509,7 @@ struct AggregateFunctionMaxData : Data
template <typename Data>
struct AggregateFunctionAnyData : Data
{
typedef AggregateFunctionAnyData<Data> Self;
using Self = AggregateFunctionAnyData<Data>;
bool changeIfBetter(const IColumn & column, size_t row_num) { return this->changeFirstTime(column, row_num); }
bool changeIfBetter(const Self & to) { return this->changeFirstTime(to); }
@ -520,7 +520,7 @@ struct AggregateFunctionAnyData : Data
template <typename Data>
struct AggregateFunctionAnyLastData : Data
{
typedef AggregateFunctionAnyLastData<Data> Self;
using Self = AggregateFunctionAnyLastData<Data>;
bool changeIfBetter(const IColumn & column, size_t row_num) { this->change(column, row_num); return true; }
bool changeIfBetter(const Self & to) { this->change(to); return true; }

View File

@ -119,7 +119,7 @@ public:
DataTypePtr getReturnType() const override
{
return new DataTypeFloat64;
return std::make_shared<DataTypeFloat64>();
}
void setArgument(const DataTypePtr & argument)
@ -383,7 +383,7 @@ public:
DataTypePtr getReturnType() const override
{
return new DataTypeFloat64;
return std::make_shared<DataTypeFloat64>();
}
void setArgumentsImpl(const DataTypes & arguments)

View File

@ -2,8 +2,6 @@
#include <memory>
#include <Poco/SharedPtr.h>
#include <DB/Core/Row.h>
#include <DB/DataTypes/IDataType.h>
#include <DB/Common/typeid_cast.h>
@ -115,7 +113,7 @@ template <typename T>
class IAggregateFunctionHelper : public IAggregateFunction
{
protected:
typedef T Data;
using Data = T;
static Data & data(AggregateDataPtr place) { return *reinterpret_cast<Data*>(place); }
static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast<const Data*>(place); }
@ -149,8 +147,6 @@ public:
};
using Poco::SharedPtr;
typedef SharedPtr<IAggregateFunction> AggregateFunctionPtr;
using AggregateFunctionPtr = std::shared_ptr<IAggregateFunction>;
}

View File

@ -75,9 +75,9 @@ template <typename Hash = UniquesHashSetDefaultHash>
class UniquesHashSet : private HashTableAllocatorWithStackMemory<(1 << UNIQUES_HASH_SET_INITIAL_SIZE_DEGREE) * sizeof(UInt32)>
{
private:
typedef UInt64 Value_t;
typedef UInt32 HashValue_t;
typedef HashTableAllocatorWithStackMemory<(1 << UNIQUES_HASH_SET_INITIAL_SIZE_DEGREE) * sizeof(UInt32)> Allocator;
using Value_t = UInt64;
using HashValue_t = UInt32;
using Allocator = HashTableAllocatorWithStackMemory<(1 << UNIQUES_HASH_SET_INITIAL_SIZE_DEGREE) * sizeof(UInt32)>;
UInt32 m_size; /// Количество элементов
UInt8 size_degree; /// Размер таблицы в виде степени двух

View File

@ -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>;
/** Соединение с сервером БД для использования в клиенте.
@ -113,7 +116,7 @@ public:
UInt64 type;
Block block;
SharedPtr<Exception> exception;
std::unique_ptr<Exception> exception;
Progress progress;
BlockStreamProfileInfo profile_info;
@ -178,8 +181,8 @@ public:
*/
void fillBlockExtraInfo(BlockExtraInfo & info) const;
size_t outBytesCount() const { return !out.isNull() ? out->count() : 0; }
size_t inBytesCount() const { return !in.isNull() ? in->count() : 0; }
size_t outBytesCount() const { return out ? out->count() : 0; }
size_t inBytesCount() const { return in ? in->count() : 0; }
private:
String host;
@ -207,8 +210,8 @@ private:
UInt64 server_revision = 0;
Poco::Net::StreamSocket socket;
SharedPtr<ReadBuffer> in;
SharedPtr<WriteBuffer> out;
std::shared_ptr<ReadBuffer> in;
std::shared_ptr<WriteBuffer> out;
String query_id;
UInt64 compression; /// Сжимать ли данные при взаимодействии с сервером.
@ -226,11 +229,11 @@ private:
Poco::Timespan ping_timeout;
/// Откуда читать результат выполнения запроса.
SharedPtr<ReadBuffer> maybe_compressed_in;
std::shared_ptr<ReadBuffer> maybe_compressed_in;
BlockInputStreamPtr block_in;
/// Куда писать данные INSERT-а.
SharedPtr<WriteBuffer> maybe_compressed_out;
std::shared_ptr<WriteBuffer> maybe_compressed_out;
BlockOutputStreamPtr block_out;
/// логгер, создаваемый лениво, чтобы не обращаться к DNS в конструкторе
@ -263,15 +266,11 @@ private:
bool ping();
Block receiveData();
SharedPtr<Exception> receiveException();
std::unique_ptr<Exception> receiveException();
Progress receiveProgress();
BlockStreamProfileInfo receiveProfileInfo();
void initBlockInput();
};
typedef SharedPtr<Connection> ConnectionPtr;
typedef std::vector<ConnectionPtr> Connections;
}

View File

@ -8,9 +8,6 @@
namespace DB
{
using Poco::SharedPtr;
/** Интерфейс для пулов соединений.
*
* Использование (на примере обычного ConnectionPool):
@ -25,7 +22,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 +53,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 +63,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 +109,7 @@ protected:
/** Создает новый объект для помещения в пул. */
ConnectionPtr allocObject() override
{
return new Connection(
return std::make_shared<Connection>(
host, port, resolved_address,
default_database, user, password,
client_name, compression,

View File

@ -32,8 +32,8 @@ namespace ErrorCodes
class ConnectionPoolWithFailover : public PoolWithFailoverBase<IConnectionPool>, public IConnectionPool
{
public:
typedef IConnectionPool::Entry Entry;
typedef PoolWithFailoverBase<IConnectionPool> Base;
using Entry = IConnectionPool::Entry;
using Base = PoolWithFailoverBase<IConnectionPool>;
ConnectionPoolWithFailover(ConnectionPools & nested_pools_,
LoadBalancing load_balancing,

View File

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

View File

@ -47,73 +47,56 @@ namespace ErrorCodes
* определяющий, какие отдельные значения надо уничтожать, а какие - нет.
* Ясно, что этот метод имел бы существенно ненулевую цену.
*/
class ColumnAggregateFunction final : public IColumn
class ColumnAggregateFunction final : public IColumn, public std::enable_shared_from_this<ColumnAggregateFunction>
{
public:
using Container_t = PaddedPODArray<AggregateDataPtr>;
private:
Arenas arenas; /// Пулы, в которых выделены состояния агрегатных функций.
/// Пулы, в которых выделены состояния агрегатных функций.
Arenas arenas;
struct Holder
{
using Ptr = SharedPtr<Holder>;
/// Используется для уничтожения состояний и для финализации значений.
AggregateFunctionPtr func;
AggregateFunctionPtr func; /// Используется для уничтожения состояний и для финализации значений.
const Ptr src; /// Источник. Используется, если данный столбец создан из другого и использует все или часть его значений.
Container_t data; /// Массив указателей на состояния агрегатных функций, расположенных в пулах.
/// Источник. Используется (удерживает источник от уничтожения),
/// если данный столбец создан из другого и использует все или часть его значений.
const std::shared_ptr<const ColumnAggregateFunction> src;
Holder(const AggregateFunctionPtr & func_) : func(func_) {}
Holder(const Ptr & src_) : func(src_->func), src(src_) {}
~Holder()
{
IAggregateFunction * function = func;
if (!function->hasTrivialDestructor() && src.isNull())
for (auto val : data)
function->destroy(val);
}
void popBack(size_t n)
{
size_t size = data.size();
size_t new_size = size - n;
if (src.isNull())
for (size_t i = new_size; i < size; ++i)
func->destroy(data[i]);
data.resize_assume_reserved(new_size);
}
};
Holder::Ptr holder; /// NOTE Вместо этого можно было бы унаследовать IColumn от enable_shared_from_this.
/// Массив указателей на состояния агрегатных функций, расположенных в пулах.
Container_t data;
public:
/// Создать столбец на основе другого.
ColumnAggregateFunction(const ColumnAggregateFunction & src)
: arenas(src.arenas), holder(new Holder(src.holder))
ColumnAggregateFunction(const ColumnAggregateFunction & other)
: arenas(other.arenas), func(other.func), src(other.shared_from_this())
{
}
public:
ColumnAggregateFunction(const AggregateFunctionPtr & func_)
: holder(new Holder(func_))
: func(func_)
{
}
ColumnAggregateFunction(const AggregateFunctionPtr & func_, const Arenas & arenas_)
: arenas(arenas_), holder(new Holder(func_))
: arenas(arenas_), func(func_)
{
}
~ColumnAggregateFunction()
{
if (!func->hasTrivialDestructor() && src)
for (auto val : data)
func->destroy(val);
}
void set(const AggregateFunctionPtr & func_)
{
holder->func = func_;
func = func_;
}
AggregateFunctionPtr getAggregateFunction() { return holder->func; }
AggregateFunctionPtr getAggregateFunction() const { return holder->func; }
AggregateFunctionPtr getAggregateFunction() { return func; }
AggregateFunctionPtr getAggregateFunction() const { return func; }
/// Захватить владение ареной.
void addArena(ArenaPtr arena_)
@ -136,7 +119,7 @@ public:
ColumnPtr cloneEmpty() const override
{
return new ColumnAggregateFunction(holder->func, Arenas(1, new Arena));
return std::make_shared<ColumnAggregateFunction>(func, Arenas(1, std::make_shared<Arena>()));
};
Field operator[](size_t n) const override
@ -144,7 +127,7 @@ public:
Field field = String();
{
WriteBufferFromString buffer(field.get<String &>());
holder.get()->func->serialize(getData()[n], buffer);
func->serialize(getData()[n], buffer);
}
return field;
}
@ -154,7 +137,7 @@ public:
res = String();
{
WriteBufferFromString buffer(res.get<String &>());
holder.get()->func->serialize(getData()[n], buffer);
func->serialize(getData()[n], buffer);
}
}
@ -176,19 +159,19 @@ public:
/// Объединить состояние в последней строке с заданным
void insertMergeFrom(const IColumn & src, size_t n)
{
holder.get()->func.get()->merge(getData().back(), static_cast<const ColumnAggregateFunction &>(src).getData()[n]);
func->merge(getData().back(), static_cast<const ColumnAggregateFunction &>(src).getData()[n]);
}
Arena & createOrGetArena()
{
if (unlikely(arenas.empty()))
arenas.emplace_back(new Arena);
arenas.emplace_back(std::make_shared<Arena>());
return *arenas.back().get();
}
void insert(const Field & x) override
{
IAggregateFunction * function = holder.get()->func;
IAggregateFunction * function = func.get();
Arena & arena = createOrGetArena();
@ -200,7 +183,7 @@ public:
void insertDefault() override
{
IAggregateFunction * function = holder.get()->func;
IAggregateFunction * function = func.get();
Arena & arena = createOrGetArena();
@ -247,7 +230,14 @@ public:
void popBack(size_t n) override
{
holder.get()->popBack(n);
size_t size = data.size();
size_t new_size = size - n;
if (!src)
for (size_t i = new_size; i < size; ++i)
func->destroy(data[i]);
data.resize_assume_reserved(new_size);
}
ColumnPtr filter(const Filter & filter, ssize_t result_size_hint) const override
@ -256,13 +246,12 @@ public:
if (size != filter.size())
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
ColumnAggregateFunction * res_ = new ColumnAggregateFunction(*this);
ColumnPtr res = res_;
std::shared_ptr<ColumnAggregateFunction> res = std::make_shared<ColumnAggregateFunction>(*this);
if (size == 0)
return res;
auto & res_data = res_->getData();
auto & res_data = res->getData();
if (result_size_hint)
res_data.reserve(result_size_hint > 0 ? result_size_hint : size);
@ -290,12 +279,11 @@ public:
if (perm.size() < limit)
throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
ColumnAggregateFunction * res_ = new ColumnAggregateFunction(*this);
ColumnPtr res = res_;
std::shared_ptr<ColumnAggregateFunction> res = std::make_shared<ColumnAggregateFunction>(*this);
res_->getData().resize(limit);
res->getData().resize(limit);
for (size_t i = 0; i < limit; ++i)
res_->getData()[i] = getData()[perm[i]];
res->getData()[i] = getData()[perm[i]];
return res;
}
@ -326,12 +314,12 @@ public:
/** Более эффективные методы манипуляции */
Container_t & getData()
{
return holder.get()->data;
return data;
}
const Container_t & getData() const
{
return holder.get()->data;
return data;
}
};

View File

@ -2,8 +2,6 @@
#include <string.h> // memcpy
#include <Poco/SharedPtr.h>
#include <DB/Common/Exception.h>
#include <DB/Common/Arena.h>
@ -22,8 +20,6 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}
using Poco::SharedPtr;
/** Cтолбeц значений типа массив.
* В памяти он представлен, как один столбец вложенного типа, размер которого равен сумме размеров всех массивов,
* и как массив смещений в нём, который позволяет достать каждый элемент.
@ -32,7 +28,7 @@ class ColumnArray final : public IColumn
{
public:
/** По индексу i находится смещение до начала i + 1 -го элемента. */
typedef ColumnVector<Offset_t> ColumnOffsets_t;
using ColumnOffsets_t = ColumnVector<Offset_t>;
/** Создать пустой столбец массивов, с типом значений, как в столбце nested_column */
explicit ColumnArray(ColumnPtr nested_column, ColumnPtr offsets_column = nullptr)
@ -40,12 +36,12 @@ public:
{
if (!offsets_column)
{
offsets = new ColumnOffsets_t;
offsets = std::make_shared<ColumnOffsets_t>();
}
else
{
if (!typeid_cast<ColumnOffsets_t *>(&*offsets_column))
throw Exception("offsets_column must be a ColumnVector<UInt64>", ErrorCodes::ILLEGAL_COLUMN);
throw Exception("offsets_column must be a ColumnUInt64", ErrorCodes::ILLEGAL_COLUMN);
}
}
@ -53,7 +49,7 @@ public:
ColumnPtr cloneEmpty() const override
{
return new ColumnArray(getData().cloneEmpty());
return std::make_shared<ColumnArray>(getData().cloneEmpty());
}
size_t size() const override
@ -294,7 +290,7 @@ public:
else
new_offsets = offsets;
return new ColumnArray(new_data, new_offsets);
return std::make_shared<ColumnArray>(new_data, new_offsets);
}
private:

View File

@ -1,7 +1,5 @@
#pragma once
#include <Poco/SharedPtr.h>
#include <DB/Core/Field.h>
#include <DB/Common/Exception.h>
#include <DB/Columns/ColumnVector.h>
@ -21,8 +19,6 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
using Poco::SharedPtr;
class IColumnConst : public IColumn
{
@ -57,11 +53,11 @@ namespace ColumnConstDetails
/** Столбец-константа может содержать внутри себя само значение,
* или, в случае массивов, SharedPtr от значения-массива,
* или, в случае массивов, std::shared_ptr от значения-массива,
* чтобы избежать проблем производительности при копировании очень больших массивов.
*
* T - тип значения,
* DataHolder - как значение хранится в таблице (либо T, либо SharedPtr<T>)
* DataHolder - как значение хранится в таблице (либо T, либо std::shared_ptr<T>)
* Derived должен реализовать методы getDataFromHolderImpl - получить ссылку на значение из holder-а.
*
* Для строк и массивов реализации sizeOfField и byteSize могут быть некорректными.
@ -81,14 +77,14 @@ protected:
: s(s_), data(data_), data_type(data_type_) {}
public:
typedef T Type;
typedef typename NearestFieldType<T>::Type FieldType;
using Type = T;
using FieldType = typename NearestFieldType<T>::Type;
std::string getName() const override { return "ColumnConst<" + TypeName<T>::get() + ">"; }
bool isNumeric() const override { return IsNumber<T>::value; }
bool isFixed() const override { return IsNumber<T>::value; }
size_t sizeOfField() const override { return sizeof(T); }
ColumnPtr cloneResized(size_t s_) const override { return new Derived(s_, data, data_type); }
ColumnPtr cloneResized(size_t s_) const override { return std::make_shared<Derived>(s_, data, data_type); }
size_t size() const override { return s; }
Field operator[](size_t n) const override { return FieldType(getDataFromHolder()); }
void get(size_t n, Field & res) const override { res = FieldType(getDataFromHolder()); }
@ -145,7 +141,7 @@ public:
if (s != filt.size())
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
return new Derived(countBytesInFilter(filt), data, data_type);
return std::make_shared<Derived>(countBytesInFilter(filt), data, data_type);
}
ColumnPtr replicate(const Offsets_t & offsets) const override
@ -154,7 +150,7 @@ public:
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
size_t replicated_size = 0 == s ? 0 : offsets.back();
return new Derived(replicated_size, data, data_type);
return std::make_shared<Derived>(replicated_size, data, data_type);
}
size_t byteSize() const override { return sizeof(data) + sizeof(s); }
@ -169,7 +165,7 @@ public:
if (perm.size() < limit)
throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
return new Derived(limit, data, data_type);
return std::make_shared<Derived>(limit, data, data_type);
}
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override
@ -232,10 +228,10 @@ public:
template <>
class ColumnConst<Array> final : public ColumnConstBase<Array, SharedPtr<Array>, ColumnConst<Array>>
class ColumnConst<Array> final : public ColumnConstBase<Array, std::shared_ptr<Array>, ColumnConst<Array>>
{
private:
friend class ColumnConstBase<Array, SharedPtr<Array>, ColumnConst<Array>>;
friend class ColumnConstBase<Array, std::shared_ptr<Array>, ColumnConst<Array>>;
Array & getDataFromHolderImpl() { return *data; }
const Array & getDataFromHolderImpl() const { return *data; }
@ -243,10 +239,10 @@ private:
public:
/// data_type_ должен быть ненулевым.
ColumnConst(size_t s_, const Array & data_, DataTypePtr data_type_ = DataTypePtr())
: ColumnConstBase<Array, SharedPtr<Array>, ColumnConst<Array>>(s_, new Array(data_), data_type_) {}
: ColumnConstBase<Array, std::shared_ptr<Array>, ColumnConst<Array>>(s_, std::make_shared<Array>(data_), data_type_) {}
ColumnConst(size_t s_, const SharedPtr<Array> & data_, DataTypePtr data_type_ = DataTypePtr())
: ColumnConstBase<Array, SharedPtr<Array>, ColumnConst<Array>>(s_, data_, data_type_) {}
ColumnConst(size_t s_, const std::shared_ptr<Array> & data_, DataTypePtr data_type_ = DataTypePtr())
: ColumnConstBase<Array, std::shared_ptr<Array>, ColumnConst<Array>>(s_, data_, data_type_) {}
StringRef getDataAt(size_t n) const override;
StringRef getDataAtWithTerminatingZero(size_t n) const override;
@ -267,10 +263,10 @@ public:
template <>
class ColumnConst<Tuple> final : public ColumnConstBase<Tuple, SharedPtr<Tuple>, ColumnConst<Tuple>>
class ColumnConst<Tuple> final : public ColumnConstBase<Tuple, std::shared_ptr<Tuple>, ColumnConst<Tuple>>
{
private:
friend class ColumnConstBase<Tuple, SharedPtr<Tuple>, ColumnConst<Tuple>>;
friend class ColumnConstBase<Tuple, std::shared_ptr<Tuple>, ColumnConst<Tuple>>;
Tuple & getDataFromHolderImpl() { return *data; }
const Tuple & getDataFromHolderImpl() const { return *data; }
@ -278,10 +274,10 @@ private:
public:
/// data_type_ должен быть ненулевым.
ColumnConst(size_t s_, const Tuple & data_, DataTypePtr data_type_ = DataTypePtr())
: ColumnConstBase<Tuple, SharedPtr<Tuple>, ColumnConst<Tuple>>(s_, new Tuple(data_), data_type_) {}
: ColumnConstBase<Tuple, std::shared_ptr<Tuple>, ColumnConst<Tuple>>(s_, std::make_shared<Tuple>(data_), data_type_) {}
ColumnConst(size_t s_, const SharedPtr<Tuple> & data_, DataTypePtr data_type_ = DataTypePtr())
: ColumnConstBase<Tuple, SharedPtr<Tuple>, ColumnConst<Tuple>>(s_, data_, data_type_) {}
ColumnConst(size_t s_, const std::shared_ptr<Tuple> & data_, DataTypePtr data_type_ = DataTypePtr())
: ColumnConstBase<Tuple, std::shared_ptr<Tuple>, ColumnConst<Tuple>>(s_, data_, data_type_) {}
StringRef getDataAt(size_t n) const override;
StringRef getDataAtWithTerminatingZero(size_t n) const override;
@ -297,16 +293,15 @@ public:
};
typedef ColumnConst<String> ColumnConstString;
typedef ColumnConst<Array> ColumnConstArray;
typedef ColumnConst<Tuple> ColumnConstTuple;
using ColumnConstString = ColumnConst<String>;
using ColumnConstArray = ColumnConst<Array>;
using ColumnConstTuple = ColumnConst<Tuple>;
template <typename T> ColumnPtr ColumnConst<T>::convertToFullColumn() const
{
ColumnVector<T> * res_ = new ColumnVector<T>;
ColumnPtr res = res_;
res_->getData().assign(this->s, this->data);
std::shared_ptr<ColumnVector<T>> res = std::make_shared<ColumnVector<T>>();
res->getData().assign(this->s, this->data);
return res;
}

View File

@ -27,11 +27,11 @@ namespace ErrorCodes
class ColumnFixedString final : public IColumn
{
public:
typedef PaddedPODArray<UInt8> Chars_t;
using Chars_t = PaddedPODArray<UInt8>;
private:
/// Байты строк, уложенные подряд. Строки хранятся без завершающего нулевого байта.
/** NOTE Требуется, чтобы смещение и тип chars в объекте был таким же, как у data в ColumnVector<UInt8>.
/** NOTE Требуется, чтобы смещение и тип chars в объекте был таким же, как у data в ColumnUInt8.
* Это используется в функции packFixed (AggregationCommon.h)
*/
Chars_t chars;
@ -46,7 +46,7 @@ public:
ColumnPtr cloneEmpty() const override
{
return new ColumnFixedString(n);
return std::make_shared<ColumnFixedString>(n);
}
size_t size() const override
@ -210,19 +210,18 @@ public:
if (col_size != filt.size())
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
ColumnFixedString * res_ = new ColumnFixedString(n);
ColumnPtr res = res_;
std::shared_ptr<ColumnFixedString> res = std::make_shared<ColumnFixedString>(n);
if (result_size_hint)
res_->chars.reserve(result_size_hint > 0 ? result_size_hint * n : chars.size());
res->chars.reserve(result_size_hint > 0 ? result_size_hint * n : chars.size());
size_t offset = 0;
for (size_t i = 0; i < col_size; ++i, offset += n)
{
if (filt[i])
{
res_->chars.resize(res_->chars.size() + n);
memcpySmallAllowReadWriteOverflow15(&res_->chars[res_->chars.size() - n], &chars[offset], n);
res->chars.resize(res->chars.size() + n);
memcpySmallAllowReadWriteOverflow15(&res->chars[res->chars.size() - n], &chars[offset], n);
}
}
@ -242,12 +241,11 @@ public:
throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
if (limit == 0)
return new ColumnFixedString(n);
return std::make_shared<ColumnFixedString>(n);
ColumnFixedString * res_ = new ColumnFixedString(n);
ColumnPtr res = res_;
std::shared_ptr<ColumnFixedString> res = std::make_shared<ColumnFixedString>(n);
Chars_t & res_chars = res_->chars;
Chars_t & res_chars = res->chars;
res_chars.resize(n * limit);
@ -264,13 +262,12 @@ public:
if (col_size != offsets.size())
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
ColumnFixedString * res_ = new ColumnFixedString(n);
ColumnPtr res = res_;
std::shared_ptr<ColumnFixedString> res = std::make_shared<ColumnFixedString>(n);
if (0 == col_size)
return res;
Chars_t & res_chars = res_->chars;
Chars_t & res_chars = res->chars;
res_chars.reserve(n * offsets.back());
Offset_t prev_offset = 0;

View File

@ -20,7 +20,7 @@ public:
bool isConst() const override { return false; }
std::string getName() const override { return "ColumnSet"; }
ColumnPtr cloneDummy(size_t s_) const override { return new ColumnSet(s_, data); }
ColumnPtr cloneDummy(size_t s_) const override { return std::make_shared<ColumnSet>(s_, data); }
ConstSetPtr getData() const { return data; }

View File

@ -27,7 +27,7 @@ namespace ErrorCodes
class ColumnString final : public IColumn
{
public:
typedef PaddedPODArray<UInt8> Chars_t;
using Chars_t = PaddedPODArray<UInt8>;
private:
/// По индексу i находится смещение до начала i + 1 -го элемента.
@ -59,7 +59,7 @@ public:
ColumnPtr cloneEmpty() const override
{
return new ColumnString;
return std::make_shared<ColumnString>();
}
Field operator[](size_t n) const override
@ -195,16 +195,15 @@ public:
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override
{
if (offsets.size() == 0)
return new ColumnString;
return std::make_shared<ColumnString>();
auto res = new ColumnString;
ColumnPtr res_{res};
auto res = std::make_shared<ColumnString>();
Chars_t & res_chars = res->chars;
Offsets_t & res_offsets = res->offsets;
filterArraysImpl<UInt8>(chars, offsets, res_chars, res_offsets, filt, result_size_hint);
return res_;
return res;
}
ColumnPtr permute(const Permutation & perm, size_t limit) const override
@ -220,13 +219,12 @@ public:
throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
if (limit == 0)
return new ColumnString;
return std::make_shared<ColumnString>();
ColumnString * res_ = new ColumnString;
ColumnPtr res = res_;
std::shared_ptr<ColumnString> res = std::make_shared<ColumnString>();
Chars_t & res_chars = res_->chars;
Offsets_t & res_offsets = res_->offsets;
Chars_t & res_chars = res->chars;
Offsets_t & res_offsets = res->offsets;
if (limit == size)
res_chars.resize(chars.size());
@ -378,14 +376,13 @@ public:
if (col_size != replicate_offsets.size())
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
ColumnString * res_ = new ColumnString;
ColumnPtr res = res_;
std::shared_ptr<ColumnString> res = std::make_shared<ColumnString>();
if (0 == col_size)
return res;
Chars_t & res_chars = res_->chars;
Offsets_t & res_offsets = res_->offsets;
Chars_t & res_chars = res->chars;
Offsets_t & res_offsets = res->offsets;
res_chars.reserve(chars.size() / col_size * replicate_offsets.back());
res_offsets.reserve(replicate_offsets.back());

View File

@ -36,9 +36,9 @@ public:
std::string getName() const override { return "Tuple"; }
SharedPtr<IColumn> cloneEmpty() const override
ColumnPtr cloneEmpty() const override
{
return new ColumnTuple(data.cloneEmpty());
return std::make_shared<ColumnTuple>(data.cloneEmpty());
}
size_t size() const override
@ -139,7 +139,7 @@ public:
for (size_t i = 0; i < columns.size(); ++i)
res_block.unsafeGetByPosition(i).column = data.unsafeGetByPosition(i).column->filter(filt, result_size_hint);
return new ColumnTuple(res_block);
return std::make_shared<ColumnTuple>(res_block);
}
ColumnPtr permute(const Permutation & perm, size_t limit) const override
@ -149,7 +149,7 @@ public:
for (size_t i = 0; i < columns.size(); ++i)
res_block.unsafeGetByPosition(i).column = data.unsafeGetByPosition(i).column->permute(perm, limit);
return new ColumnTuple(res_block);
return std::make_shared<ColumnTuple>(res_block);
}
ColumnPtr replicate(const Offsets_t & offsets) const override
@ -159,7 +159,7 @@ public:
for (size_t i = 0; i < columns.size(); ++i)
res_block.unsafeGetByPosition(i).column = data.unsafeGetByPosition(i).column->replicate(offsets);
return new ColumnTuple(res_block);
return std::make_shared<ColumnTuple>(res_block);
}
int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override
@ -258,7 +258,7 @@ public:
if (auto converted = materialized.unsafeGetByPosition(i).column->convertToFullColumnIfConst())
materialized.unsafeGetByPosition(i).column = converted;
return new ColumnTuple(materialized);
return std::make_shared<ColumnTuple>(materialized);
}

View File

@ -125,10 +125,10 @@ template <typename T>
class ColumnVector final : public IColumn
{
private:
typedef ColumnVector<T> Self;
using Self = ColumnVector<T>;
public:
typedef T value_type;
typedef PaddedPODArray<value_type> Container_t;
using value_type = T;
using Container_t = PaddedPODArray<value_type>;
ColumnVector() {}
ColumnVector(const size_t n) : data{n} {}
@ -246,7 +246,7 @@ public:
ColumnPtr cloneEmpty() const override
{
return new ColumnVector<T>;
return std::make_shared<ColumnVector<T>>();
}
Field operator[](size_t n) const override
@ -291,9 +291,8 @@ public:
if (size != filt.size())
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
Self * res_ = new Self;
ColumnPtr res = res_;
typename Self::Container_t & res_data = res_->getData();
std::shared_ptr<Self> res = std::make_shared<Self>();
typename Self::Container_t & res_data = res->getData();
if (result_size_hint)
res_data.reserve(result_size_hint > 0 ? result_size_hint : size);
@ -361,9 +360,8 @@ public:
if (perm.size() < limit)
throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
Self * res_ = new Self(limit);
ColumnPtr res = res_;
typename Self::Container_t & res_data = res_->getData();
std::shared_ptr<Self> res = std::make_shared<Self>(limit);
typename Self::Container_t & res_data = res->getData();
for (size_t i = 0; i < limit; ++i)
res_data[i] = data[perm[i]];
@ -377,11 +375,10 @@ public:
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
if (0 == size)
return new Self;
return std::make_shared<Self>();
Self * res_ = new Self;
ColumnPtr res = res_;
typename Self::Container_t & res_data = res_->getData();
std::shared_ptr<Self> res = std::make_shared<Self>();
typename Self::Container_t & res_data = res->getData();
res_data.reserve(offsets.back());
IColumn::Offset_t prev_offset = 0;

View File

@ -10,31 +10,31 @@ namespace DB
/** Столбцы чисел. */
typedef ColumnVector<UInt8> ColumnUInt8;
typedef ColumnVector<UInt16> ColumnUInt16;
typedef ColumnVector<UInt32> ColumnUInt32;
typedef ColumnVector<UInt64> ColumnUInt64;
using ColumnUInt8 = ColumnVector<UInt8>;
using ColumnUInt16 = ColumnVector<UInt16>;
using ColumnUInt32 = ColumnVector<UInt32>;
using ColumnUInt64 = ColumnVector<UInt64>;
typedef ColumnVector<Int8> ColumnInt8;
typedef ColumnVector<Int16> ColumnInt16;
typedef ColumnVector<Int32> ColumnInt32;
typedef ColumnVector<Int64> ColumnInt64;
using ColumnInt8 = ColumnVector<Int8>;
using ColumnInt16 = ColumnVector<Int16>;
using ColumnInt32 = ColumnVector<Int32>;
using ColumnInt64 = ColumnVector<Int64>;
typedef ColumnVector<Float32> ColumnFloat32;
typedef ColumnVector<Float64> ColumnFloat64;
using ColumnFloat32 = ColumnVector<Float32>;
using ColumnFloat64 = ColumnVector<Float64>;
typedef ColumnConst<UInt8> ColumnConstUInt8;
typedef ColumnConst<UInt16> ColumnConstUInt16;
typedef ColumnConst<UInt32> ColumnConstUInt32;
typedef ColumnConst<UInt64> ColumnConstUInt64;
using ColumnConstUInt8 = ColumnConst<UInt8>;
using ColumnConstUInt16 = ColumnConst<UInt16>;
using ColumnConstUInt32 = ColumnConst<UInt32>;
using ColumnConstUInt64 = ColumnConst<UInt64>;
typedef ColumnConst<Int8> ColumnConstInt8;
typedef ColumnConst<Int16> ColumnConstInt16;
typedef ColumnConst<Int32> ColumnConstInt32;
typedef ColumnConst<Int64> ColumnConstInt64;
using ColumnConstInt8 = ColumnConst<Int8>;
using ColumnConstInt16 = ColumnConst<Int16>;
using ColumnConstInt32 = ColumnConst<Int32>;
using ColumnConstInt64 = ColumnConst<Int64>;
typedef ColumnConst<Float32> ColumnConstFloat32;
typedef ColumnConst<Float64> ColumnConstFloat64;
using ColumnConstFloat32 = ColumnConst<Float32>;
using ColumnConstFloat64 = ColumnConst<Float64>;
}

View File

@ -1,6 +1,6 @@
#pragma once
#include <Poco/SharedPtr.h>
#include <memory>
#include <DB/Common/PODArray.h>
#include <DB/Common/typeid_cast.h>
@ -19,11 +19,9 @@ namespace ErrorCodes
extern const int NOT_IMPLEMENTED;
}
using Poco::SharedPtr;
class IColumn;
using ColumnPtr = SharedPtr<IColumn>;
using ColumnPtr = std::shared_ptr<IColumn>;
using Columns = std::vector<ColumnPtr>;
using ColumnPlainPtrs = std::vector<IColumn *>;
using ConstColumnPlainPtrs = std::vector<const IColumn *>;
@ -55,7 +53,7 @@ public:
* и он может содержать как константные, так и полноценные столбцы,
* то превратить в нём все константные столбцы в полноценные, и вернуть результат.
*/
virtual SharedPtr<IColumn> convertToFullColumnIfConst() const { return {}; }
virtual ColumnPtr convertToFullColumnIfConst() const { return {}; }
/** Значения имеют фиксированную длину.
*/
@ -66,16 +64,16 @@ public:
virtual size_t sizeOfField() const { throw Exception("Cannot get sizeOfField() for column " + getName(), ErrorCodes::CANNOT_GET_SIZE_OF_FIELD); }
/** Создать столбец с такими же данными. */
virtual SharedPtr<IColumn> clone() const { return cut(0, size()); }
virtual ColumnPtr clone() const { return cut(0, size()); }
/** Создать пустой столбец такого же типа */
virtual SharedPtr<IColumn> cloneEmpty() const { return cloneResized(0); }
virtual ColumnPtr cloneEmpty() const { return cloneResized(0); }
/** Создать столбец такого же типа и указанного размера.
* Если размер меньше текущего, данные обрезаются.
* Если больше - добавляются значения по умолчанию.
*/
virtual SharedPtr<IColumn> cloneResized(size_t size) const { throw Exception("Cannot cloneResized() column " + getName(), ErrorCodes::NOT_IMPLEMENTED); }
virtual ColumnPtr cloneResized(size_t size) const { throw Exception("Cannot cloneResized() column " + getName(), ErrorCodes::NOT_IMPLEMENTED); }
/** Количество значений в столбце. */
virtual size_t size() const = 0;
@ -117,9 +115,9 @@ public:
/** Удалить всё кроме диапазона элементов.
* Используется, например, для операции LIMIT.
*/
virtual SharedPtr<IColumn> cut(size_t start, size_t length) const
virtual ColumnPtr cut(size_t start, size_t length) const
{
SharedPtr<IColumn> res = cloneEmpty();
ColumnPtr res = cloneEmpty();
res.get()->insertRangeFrom(*this, start, length);
return res;
}
@ -193,15 +191,15 @@ public:
* если 0, то не делать reserve,
* иначе сделать reserve по размеру исходного столбца.
*/
typedef PaddedPODArray<UInt8> Filter;
virtual SharedPtr<IColumn> filter(const Filter & filt, ssize_t result_size_hint) const = 0;
using Filter = PaddedPODArray<UInt8>;
virtual ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const = 0;
/** Переставить значения местами, используя указанную перестановку.
* Используется при сортировке.
* limit - если не равно 0 - положить в результат только первые limit значений.
*/
typedef PaddedPODArray<size_t> Permutation;
virtual SharedPtr<IColumn> permute(const Permutation & perm, size_t limit) const = 0;
using Permutation = PaddedPODArray<size_t>;
virtual ColumnPtr permute(const Permutation & perm, size_t limit) const = 0;
/** Сравнить (*this)[n] и rhs[m].
* Вернуть отрицательное число, 0, или положительное число, если меньше, равно, или больше, соответственно.
@ -229,9 +227,9 @@ public:
* (i-е значение размножается в offsets[i] - offsets[i - 1] значений.)
* Необходимо для реализации операции ARRAY JOIN.
*/
typedef UInt64 Offset_t;
typedef PaddedPODArray<Offset_t> Offsets_t;
virtual SharedPtr<IColumn> replicate(const Offsets_t & offsets) const = 0;
using Offset_t = UInt64;
using Offsets_t = PaddedPODArray<Offset_t>;
virtual ColumnPtr replicate(const Offsets_t & offsets) const = 0;
/** Посчитать минимум и максимум по столбцу.
* Функция должна быть реализована полноценно только для числовых столбцов, а также дат/дат-с-временем.

View File

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

View File

@ -150,8 +150,8 @@ public:
return data + sizeof(T) * i;
}
typedef T * iterator;
typedef const T * const_iterator;
using iterator = T *;
using const_iterator = const T *;
iterator begin() { return &elem(0); }
iterator end() { return &elem(size()); }

View File

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

View File

@ -14,8 +14,8 @@
#include <Poco/Util/AbstractConfiguration.h>
#include <common/logger_useful.h>
typedef Poco::AutoPtr<Poco::Util::AbstractConfiguration> ConfigurationPtr;
typedef Poco::AutoPtr<Poco::XML::Document> XMLDocumentPtr;
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
using XMLDocumentPtr = Poco::AutoPtr<Poco::XML::Document>;
class ConfigProcessor
{
@ -50,8 +50,8 @@ private:
bool throw_on_bad_incl;
Substitutions substitutions;
typedef XMLDocumentPtr DocumentPtr;
typedef Poco::AutoPtr<Poco::XML::Node> NodePtr;
using DocumentPtr = XMLDocumentPtr;
using NodePtr = Poco::AutoPtr<Poco::XML::Node>;
void mergeRecursive(DocumentPtr config, Poco::XML::Node * config_node, Poco::XML::Node * with_node);

View File

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

View File

@ -2,9 +2,9 @@
#include <cerrno>
#include <vector>
#include <memory>
#include <Poco/Exception.h>
#include <Poco/SharedPtr.h>
#include <DB/Common/StackTrace.h>
@ -67,7 +67,7 @@ private:
};
typedef std::vector<std::exception_ptr> Exceptions;
using Exceptions = std::vector<std::exception_ptr>;
void throwFromErrno(const std::string & s, int code = 0, int the_errno = errno);
@ -90,7 +90,7 @@ std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace);
void rethrowFirstException(Exceptions & exceptions);
Poco::SharedPtr<Poco::Exception> convertCurrentException();
std::unique_ptr<Poco::Exception> convertCurrentException();
template <typename T>

View File

@ -59,7 +59,7 @@ public:
{
initReadBuffer();
initSampleBlock(context);
ExternalTableData res = std::make_pair(new AsynchronousBlockInputStream(context.getInputFormat(
ExternalTableData res = std::make_pair(std::make_shared<AsynchronousBlockInputStream>(context.getInputFormat(
format, *read_buffer, sample_block, DEFAULT_BLOCK_SIZE)), name);
return res;
}
@ -169,7 +169,7 @@ public:
void handlePart(const Poco::Net::MessageHeader & header, std::istream & stream)
{
/// Буфер инициализируется здесь, а не в виртуальной функции initReadBuffer
read_buffer.reset(new ReadBufferFromIStream(stream));
read_buffer = std::make_unique<ReadBufferFromIStream>(stream);
/// Извлекаем коллекцию параметров из MessageHeader
Poco::Net::NameValueCollection content;
@ -190,7 +190,7 @@ public:
ExternalTableData data = getData(context);
/// Создаем таблицу
NamesAndTypesListPtr columns = new NamesAndTypesList(sample_block.getColumnsList());
NamesAndTypesListPtr columns = std::make_shared<NamesAndTypesList>(sample_block.getColumnsList());
StoragePtr storage = StorageMemory::create(data.second, columns);
context.addExternalTable(data.second, storage);
BlockOutputStreamPtr output = storage->write(ASTPtr(), context.getSettingsRef());

View File

@ -26,9 +26,9 @@ template
class ClearableHashMap : public HashTable<Key, ClearableHashMapCell<Key, Mapped, Hash>, Hash, Grower, Allocator>
{
public:
typedef Key key_type;
typedef Mapped mapped_type;
typedef typename ClearableHashMap::cell_type::value_type value_type;
using key_type = Key;
using mapped_type = Mapped;
using value_type = typename ClearableHashMap::cell_type::value_type;
mapped_type & operator[](Key x)
{

View File

@ -30,8 +30,8 @@ struct ClearableHashSetState
template <typename Key, typename BaseCell>
struct ClearableHashTableCell : public BaseCell
{
typedef ClearableHashSetState State;
typedef typename BaseCell::value_type value_type;
using State = ClearableHashSetState;
using value_type = typename BaseCell::value_type;
UInt32 version;
@ -59,8 +59,8 @@ template
class ClearableHashSet : public HashTable<Key, ClearableHashTableCell<Key, HashTableCell<Key, Hash, ClearableHashSetState>>, Hash, Grower, Allocator>
{
public:
typedef Key key_type;
typedef typename ClearableHashSet::cell_type::value_type value_type;
using key_type = Key;
using value_type = typename ClearableHashSet::cell_type::value_type;
void clear()
{

View File

@ -29,10 +29,10 @@ struct PairNoInit
template <typename Key, typename TMapped, typename Hash, typename TState = HashTableNoState>
struct HashMapCell
{
typedef TMapped Mapped;
typedef TState State;
using Mapped = TMapped;
using State = TState;
typedef PairNoInit<Key, Mapped> value_type;
using value_type = PairNoInit<Key, Mapped>;
value_type value;
HashMapCell() {}
@ -98,7 +98,7 @@ struct HashMapCell
template <typename Key, typename TMapped, typename Hash, typename TState = HashTableNoState>
struct HashMapCellWithSavedHash : public HashMapCell<Key, TMapped, Hash, TState>
{
typedef HashMapCell<Key, TMapped, Hash, TState> Base;
using Base = HashMapCell<Key, TMapped, Hash, TState>;
size_t saved_hash;
@ -123,9 +123,9 @@ template
class HashMapTable : public HashTable<Key, Cell, Hash, Grower, Allocator>
{
public:
typedef Key key_type;
typedef typename Cell::Mapped mapped_type;
typedef typename Cell::value_type value_type;
using key_type = Key;
using mapped_type = typename Cell::Mapped;
using value_type = typename Cell::value_type;
using HashTable<Key, Cell, Hash, Grower, Allocator>::HashTable;

View File

@ -22,8 +22,8 @@ template
class HashSetTable : public HashTable<Key, TCell, Hash, Grower, Allocator>
{
public:
typedef HashSetTable<Key, TCell, Hash, Grower, Allocator> Self;
typedef TCell Cell;
using Self = HashSetTable<Key, TCell, Hash, Grower, Allocator>;
using Cell = TCell;
void merge(const Self & rhs)
{
@ -61,7 +61,7 @@ public:
template <typename Key, typename Hash, typename TState = HashTableNoState>
struct HashSetCellWithSavedHash : public HashTableCell<Key, Hash, TState>
{
typedef HashTableCell<Key, Hash, TState> Base;
using Base = HashTableCell<Key, Hash, TState>;
size_t saved_hash;

View File

@ -82,9 +82,9 @@ void set(T & x) { x = 0; }
template <typename Key, typename Hash, typename TState = HashTableNoState>
struct HashTableCell
{
typedef TState State;
using State = TState;
typedef Key value_type;
using value_type = Key;
Key key;
HashTableCell() {}
@ -263,9 +263,9 @@ protected:
template <typename, typename, typename, typename, typename, typename, size_t>
friend class TwoLevelHashTable;
typedef size_t HashValue;
typedef HashTable<Key, Cell, Hash, Grower, Allocator> Self;
typedef Cell cell_type;
using HashValue = size_t;
using Self = HashTable<Key, Cell, Hash, Grower, Allocator>;
using cell_type = Cell;
size_t m_size = 0; /// Количество элементов
Cell * buf; /// Кусок памяти для всех элементов кроме элемента с ключём 0.
@ -419,8 +419,8 @@ protected:
public:
typedef Key key_type;
typedef typename Cell::value_type value_type;
using key_type = Key;
using value_type = typename Cell::value_type;
size_t hash(const Key & x) const { return Hash::operator()(x); }

View File

@ -29,8 +29,8 @@ protected:
friend class iterator;
friend class Reader;
typedef SmallTable<Key, Cell, capacity> Self;
typedef Cell cell_type;
using Self = SmallTable<Key, Cell, capacity>;
using cell_type = Cell;
size_t m_size = 0; /// Количество элементов.
Cell buf[capacity]; /// Кусок памяти для всех элементов.
@ -63,8 +63,8 @@ protected:
public:
typedef Key key_type;
typedef typename Cell::value_type value_type;
using key_type = Key;
using value_type = typename Cell::value_type;
class Reader final : private Cell::State
@ -382,9 +382,9 @@ template
class SmallMapTable : public SmallTable<Key, Cell, capacity>
{
public:
typedef Key key_type;
typedef typename Cell::Mapped mapped_type;
typedef typename Cell::value_type value_type;
using key_type = Key;
using mapped_type = typename Cell::Mapped;
using value_type = typename Cell::value_type;
mapped_type & ALWAYS_INLINE operator[](Key x)
{

View File

@ -15,9 +15,9 @@ template
class TwoLevelHashMapTable : public TwoLevelHashTable<Key, Cell, Hash, Grower, Allocator, HashMapTable<Key, Cell, Hash, Grower, Allocator>>
{
public:
typedef Key key_type;
typedef typename Cell::Mapped mapped_type;
typedef typename Cell::value_type value_type;
using key_type = Key;
using mapped_type = typename Cell::Mapped;
using value_type = typename Cell::value_type;
using TwoLevelHashTable<Key, Cell, Hash, Grower, Allocator, HashMapTable<Key, Cell, Hash, Grower, Allocator> >::TwoLevelHashTable;

View File

@ -42,10 +42,10 @@ protected:
friend class const_iterator;
friend class iterator;
typedef size_t HashValue;
typedef TwoLevelHashTable<Key, Cell, Hash, Grower, Allocator, ImplTable> Self;
using HashValue = size_t;
using Self = TwoLevelHashTable<Key, Cell, Hash, Grower, Allocator, ImplTable>;
public:
typedef ImplTable Impl;
using Impl = ImplTable;
static constexpr size_t NUM_BUCKETS = 1 << BITS_FOR_BUCKET;
static constexpr size_t MAX_BUCKET = NUM_BUCKETS - 1;
@ -81,8 +81,8 @@ protected:
}
public:
typedef typename Impl::key_type key_type;
typedef typename Impl::value_type value_type;
using key_type = typename Impl::key_type;
using value_type = typename Impl::value_type;
Impl impls[NUM_BUCKETS];

View File

@ -53,10 +53,10 @@ private:
};
template<UInt8 K> struct MinCounterTypeHelper;
template<> struct MinCounterTypeHelper<0> { typedef UInt8 Type; };
template<> struct MinCounterTypeHelper<1> { typedef UInt16 Type; };
template<> struct MinCounterTypeHelper<2> { typedef UInt32 Type; };
template<> struct MinCounterTypeHelper<3> { typedef UInt64 Type; };
template<> struct MinCounterTypeHelper<0> { using Type = UInt8; };
template<> struct MinCounterTypeHelper<1> { using Type = UInt16; };
template<> struct MinCounterTypeHelper<2> { using Type = UInt32; };
template<> struct MinCounterTypeHelper<3> { using Type = UInt64; };
/// Вспомогательная структура для автоматического определения
/// минимального размера типа счетчика в зависимости от максимального значения.

View File

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

View File

@ -21,7 +21,7 @@ public:
String expand(const String & s, size_t level = 0) const;
private:
typedef std::map<String, String> MacroMap;
using MacroMap = std::map<String, String>;
MacroMap macros;
};

View File

@ -4,8 +4,6 @@
#include <vector>
#include <memory>
#include <Poco/SharedPtr.h>
#include <re2/re2.h>
#include <re2_st/re2.h>
@ -47,7 +45,7 @@ public:
};
using Match = OptimizedRegularExpressionDetails::Match;
typedef std::vector<Match> MatchVec;
using MatchVec = std::vector<Match>;
using RegexType = typename std::conditional<thread_safe, re2::RE2, re2_st::RE2>::type;
using StringPieceType = typename std::conditional<thread_safe, re2::StringPiece, re2_st::StringPiece>::type;

View File

@ -110,7 +110,7 @@ private:
}
public:
typedef T value_type;
using value_type = T;
size_t allocated_size() const { return c_end_of_storage - c_start + pad_right; }

View File

@ -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;
};
@ -85,12 +84,12 @@ public:
Object & operator*() & { return *data->data.object; }
const Object & operator*() const & { return *data->data.object; }
bool isNull() const { return data.isNull(); }
bool isNull() const { return data == nullptr; }
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:

View File

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

View File

@ -29,8 +29,8 @@
class SipHash
{
private:
typedef uint64_t u64;
typedef uint8_t u8;
using u64 = uint64_t;
using u8 = uint8_t;
/// Состояние.
u64 v0;

View File

@ -17,7 +17,7 @@ public:
std::string toString() const;
private:
typedef void* Frame;
using Frame = void*;
Frame frames[STACK_TRACE_MAX_DEPTH];
size_t frames_size;
};

View File

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

View File

@ -81,6 +81,6 @@ private:
};
typedef std::shared_ptr<Throttler> ThrottlerPtr;
using ThrottlerPtr = std::shared_ptr<Throttler>;
}

View File

@ -25,9 +25,9 @@ class Context;
class Block
{
public:
typedef std::list<ColumnWithTypeAndName> Container_t;
typedef std::vector<Container_t::iterator> IndexByPosition_t;
typedef std::map<String, Container_t::iterator> IndexByName_t;
using Container_t = std::list<ColumnWithTypeAndName>;
using IndexByPosition_t = std::vector<Container_t::iterator>;
using IndexByName_t = std::map<String, Container_t::iterator>;
private:
Container_t data;
@ -123,8 +123,8 @@ public:
void swap(Block & other);
};
typedef std::vector<Block> Blocks;
typedef std::list<Block> BlocksList;
using Blocks = std::vector<Block>;
using BlocksList = std::list<Block>;
/// Сравнить типы столбцов у блоков. Порядок столбцов имеет значение. Имена не имеют значения.
bool blocksHaveEqualStructure(const Block & lhs, const Block & rhs);

View File

@ -7,6 +7,6 @@
namespace DB
{
typedef std::vector<size_t> ColumnNumbers;
using ColumnNumbers = std::vector<size_t>;
}

View File

@ -8,6 +8,6 @@
namespace DB
{
typedef std::vector<ColumnWithTypeAndName> ColumnsWithTypeAndName;
using ColumnsWithTypeAndName = std::vector<ColumnWithTypeAndName>;
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <vector>
#include <algorithm>
#include <type_traits>
#include <functional>
@ -27,8 +28,6 @@ using TupleBackend = std::vector<Field>;
STRONG_TYPEDEF(TupleBackend, Tuple); /// Значение типа "кортеж"
using Poco::SharedPtr;
/** 32 хватает с запасом (достаточно 28), но выбрано круглое число,
* чтобы арифметика при использовании массивов из Field была проще (не содержала умножения).
*/
@ -159,14 +158,14 @@ public:
template <typename T> T & get()
{
typedef typename std::remove_reference<T>::type TWithoutRef;
using TWithoutRef = typename std::remove_reference<T>::type;
TWithoutRef * __attribute__((__may_alias__)) ptr = reinterpret_cast<TWithoutRef*>(storage);
return *ptr;
};
template <typename T> const T & get() const
{
typedef typename std::remove_reference<T>::type TWithoutRef;
using TWithoutRef = typename std::remove_reference<T>::type;
const TWithoutRef * __attribute__((__may_alias__)) ptr = reinterpret_cast<const TWithoutRef*>(storage);
return *ptr;
};
@ -356,13 +355,13 @@ template <> struct Field::TypeToEnum<String> { static const Types::Which
template <> struct Field::TypeToEnum<Array> { static const Types::Which value = Types::Array; };
template <> struct Field::TypeToEnum<Tuple> { static const Types::Which value = Types::Tuple; };
template <> struct Field::EnumToType<Field::Types::Null> { typedef Null Type; };
template <> struct Field::EnumToType<Field::Types::UInt64> { typedef UInt64 Type; };
template <> struct Field::EnumToType<Field::Types::Int64> { typedef Int64 Type; };
template <> struct Field::EnumToType<Field::Types::Float64> { typedef Float64 Type; };
template <> struct Field::EnumToType<Field::Types::String> { typedef String Type; };
template <> struct Field::EnumToType<Field::Types::Array> { typedef Array Type; };
template <> struct Field::EnumToType<Field::Types::Tuple> { typedef Tuple Type; };
template <> struct Field::EnumToType<Field::Types::Null> { using Type = Null ; };
template <> struct Field::EnumToType<Field::Types::UInt64> { using Type = UInt64 ; };
template <> struct Field::EnumToType<Field::Types::Int64> { using Type = Int64 ; };
template <> struct Field::EnumToType<Field::Types::Float64> { using Type = Float64 ; };
template <> struct Field::EnumToType<Field::Types::String> { using Type = String ; };
template <> struct Field::EnumToType<Field::Types::Array> { using Type = Array ; };
template <> struct Field::EnumToType<Field::Types::Tuple> { using Type = Tuple ; };
template <typename T>
@ -396,20 +395,20 @@ template <> struct TypeName<Tuple> { static std::string get() { return "Tuple";
template <typename T> struct NearestFieldType;
template <> struct NearestFieldType<UInt8> { typedef UInt64 Type; };
template <> struct NearestFieldType<UInt16> { typedef UInt64 Type; };
template <> struct NearestFieldType<UInt32> { typedef UInt64 Type; };
template <> struct NearestFieldType<UInt64> { typedef UInt64 Type; };
template <> struct NearestFieldType<Int8> { typedef Int64 Type; };
template <> struct NearestFieldType<Int16> { typedef Int64 Type; };
template <> struct NearestFieldType<Int32> { typedef Int64 Type; };
template <> struct NearestFieldType<Int64> { typedef Int64 Type; };
template <> struct NearestFieldType<Float32> { typedef Float64 Type; };
template <> struct NearestFieldType<Float64> { typedef Float64 Type; };
template <> struct NearestFieldType<String> { typedef String Type; };
template <> struct NearestFieldType<Array> { typedef Array Type; };
template <> struct NearestFieldType<Tuple> { typedef Tuple Type; };
template <> struct NearestFieldType<bool> { typedef UInt64 Type; };
template <> struct NearestFieldType<UInt8> { using Type = UInt64 ; };
template <> struct NearestFieldType<UInt16> { using Type = UInt64 ; };
template <> struct NearestFieldType<UInt32> { using Type = UInt64 ; };
template <> struct NearestFieldType<UInt64> { using Type = UInt64 ; };
template <> struct NearestFieldType<Int8> { using Type = Int64 ; };
template <> struct NearestFieldType<Int16> { using Type = Int64 ; };
template <> struct NearestFieldType<Int32> { using Type = Int64 ; };
template <> struct NearestFieldType<Int64> { using Type = Int64 ; };
template <> struct NearestFieldType<Float32> { using Type = Float64 ; };
template <> struct NearestFieldType<Float64> { using Type = Float64 ; };
template <> struct NearestFieldType<String> { using Type = String ; };
template <> struct NearestFieldType<Array> { using Type = Array ; };
template <> struct NearestFieldType<Tuple> { using Type = Tuple ; };
template <> struct NearestFieldType<bool> { using Type = UInt64 ; };
template <typename T>

View File

@ -26,7 +26,7 @@ namespace ErrorCodes
template <typename R = void>
struct StaticVisitor
{
typedef R ResultType;
using ResultType = R;
};

View File

@ -9,8 +9,8 @@
namespace DB
{
typedef std::vector<std::string> Names;
typedef std::unordered_set<std::string> NameSet;
typedef std::unordered_map<std::string, std::string> NameToNameMap;
using Names = std::vector<std::string>;
using NameSet = std::unordered_set<std::string>;
using NameToNameMap = std::unordered_map<std::string, std::string>;
}

View File

@ -63,6 +63,6 @@ public:
NamesAndTypesList addTypes(const Names & names) const;
};
using NamesAndTypesListPtr = SharedPtr<NamesAndTypesList>;
using NamesAndTypesListPtr = std::shared_ptr<NamesAndTypesList>;
}

View File

@ -13,6 +13,6 @@ namespace DB
* Внимание! Предпочтительно вместо единичных строк хранить блоки столбцов. См. Block.h
*/
typedef AutoArray<Field> Row;
using Row = AutoArray<Field>;
}

View File

@ -18,12 +18,12 @@ struct SortColumnDescription
String column_name; /// Имя столбца.
size_t column_number; /// Номер столбца (используется, если не задано имя).
int direction; /// 1 - по возрастанию, -1 - по убыванию.
Poco::SharedPtr<Collator> collator; /// Collator для locale-specific сортировки строк
std::shared_ptr<Collator> collator; /// Collator для locale-specific сортировки строк
SortColumnDescription(size_t column_number_, int direction_, const Poco::SharedPtr<Collator> & collator_ = nullptr)
SortColumnDescription(size_t column_number_, int direction_, const std::shared_ptr<Collator> & collator_ = nullptr)
: column_number(column_number_), direction(direction_), collator(collator_) {}
SortColumnDescription(String column_name_, int direction_, const Poco::SharedPtr<Collator> & collator_ = nullptr)
SortColumnDescription(String column_name_, int direction_, const std::shared_ptr<Collator> & collator_ = nullptr)
: column_name(column_name_), column_number(0), direction(direction_), collator(collator_) {}
/// Для IBlockInputStream.
@ -31,14 +31,14 @@ struct SortColumnDescription
{
std::stringstream res;
res << column_name << ", " << column_number << ", " << direction;
if (!collator.isNull())
if (collator)
res << ", collation locale: " << collator->getLocale();
return res.str();
}
};
/// Описание правила сортировки по нескольким столбцам.
typedef std::vector<SortColumnDescription> SortDescription;
using SortDescription = std::vector<SortColumnDescription>;
/** Курсор, позволяющий сравнивать соответствующие строки в разных блоках.
@ -59,7 +59,7 @@ struct SortCursorImpl
*/
size_t order;
typedef std::vector<UInt8> NeedCollationFlags;
using NeedCollationFlags = std::vector<UInt8>;
/** Нужно ли использовать Collator для сортировки столбца */
NeedCollationFlags need_collation;
@ -96,7 +96,7 @@ struct SortCursorImpl
sort_columns.push_back(&*block.getByPosition(column_number).column);
need_collation[j] = !desc[j].collator.isNull() && sort_columns.back()->getName() == "ColumnString";
need_collation[j] = desc[j].collator != nullptr && sort_columns.back()->getName() == "ColumnString";
has_collation |= need_collation[j];
}

View File

@ -30,7 +30,7 @@ struct StringRef
explicit operator std::string() const { return toString(); }
};
typedef std::vector<StringRef> StringRefs;
using StringRefs = std::vector<StringRef>;
#if defined(__x86_64__)

View File

@ -14,21 +14,21 @@ namespace DB
STRONG_TYPEDEF(char, Null);
typedef Poco::UInt8 UInt8;
typedef Poco::UInt16 UInt16;
typedef Poco::UInt32 UInt32;
typedef Poco::UInt64 UInt64;
using UInt8 = Poco::UInt8;
using UInt16 = Poco::UInt16;
using UInt32 = Poco::UInt32;
using UInt64 = Poco::UInt64;
typedef Poco::Int8 Int8;
typedef Poco::Int16 Int16;
typedef Poco::Int32 Int32;
typedef Poco::Int64 Int64;
using Int8 = Poco::Int8;
using Int16 = Poco::Int16;
using Int32 = Poco::Int32;
using Int64 = Poco::Int64;
typedef float Float32;
typedef double Float64;
using Float32 = float;
using Float64 = double;
typedef std::string String;
typedef std::vector<String> Strings;
using String = std::string;
using Strings = std::vector<String>;
template <typename T> struct IsNumber { static const bool value = false; };

View File

@ -1,7 +1,5 @@
#pragma once
#include <Poco/SharedPtr.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/Columns/ColumnConst.h>
#include <DB/Core/ColumnWithTypeAndName.h>

View File

@ -1,7 +1,5 @@
#pragma once
#include <Poco/SharedPtr.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/Interpreters/evaluateMissingDefaults.h>
#include <DB/Columns/ColumnConst.h>

View File

@ -1,7 +1,5 @@
#pragma once
#include <Poco/SharedPtr.h>
#include <DB/DataStreams/IBlockOutputStream.h>
#include <DB/Columns/ColumnConst.h>

View File

@ -11,8 +11,6 @@
namespace DB
{
using Poco::SharedPtr;
/** Агрегирует поток блоков, используя заданные столбцы-ключи и агрегатные функции.
* Столбцы с агрегатными функциями добавляет в конец блока.
@ -58,7 +56,7 @@ protected:
BlockInputStreamPtr block_in;
TemporaryFileStream(const std::string & path)
: file_in(path), compressed_in(file_in), block_in(new NativeBlockInputStream(compressed_in, ClickHouseRevision::get())) {}
: file_in(path), compressed_in(file_in), block_in(std::make_shared<NativeBlockInputStream>(compressed_in, ClickHouseRevision::get())) {}
};
std::vector<std::unique_ptr<TemporaryFileStream>> temporary_inputs;

View File

@ -1,7 +1,5 @@
#pragma once
#include <Poco/SharedPtr.h>
#include <DB/Core/Block.h>
#include <DB/IO/WriteBuffer.h>
#include <DB/DataStreams/IRowOutputStream.h>

View File

@ -47,7 +47,7 @@ protected:
private:
struct MergingBlock;
typedef std::vector<MergingBlock*> BlockPlainPtrs;
using BlockPlainPtrs = std::vector<MergingBlock*>;
struct MergingBlock : boost::noncopyable
{
@ -229,7 +229,7 @@ private:
}
};
typedef std::priority_queue<Cursor> Queue;
using Queue = std::priority_queue<Cursor>;
const SortDescription description;
String sign_column_name;

View File

@ -6,8 +6,6 @@
namespace DB
{
using Poco::SharedPtr;
/** Объединяет несколько источников в один.
* В отличие от UnionBlockInputStream, делает это последовательно.

View File

@ -51,7 +51,7 @@ private:
size_t max_bytes;
OverflowMode overflow_mode;
typedef HashSet<UInt128, UInt128TrivialHash> SetHashed;
using SetHashed = HashSet<UInt128, UInt128TrivialHash>;
SetHashed set;
};

View File

@ -1,7 +1,5 @@
#pragma once
#include <Poco/SharedPtr.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>

View File

@ -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(std::make_shared<QueueBlockIOStream>(1));
return destinations.back();
}
@ -65,12 +63,12 @@ private:
* Сделаны на основе очереди небольшой длины.
* Блок из source кладётся в каждую очередь.
*/
typedef SharedPtr<QueueBlockIOStream> Destination;
typedef std::list<Destination> Destinations;
using Destination = std::shared_ptr<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>;
}

View File

@ -2,8 +2,6 @@
#include <boost/noncopyable.hpp>
#include <Poco/SharedPtr.h>
#include <DB/Core/Block.h>
#include <DB/Core/Progress.h>
#include <DB/Storages/IStorage.h>
@ -12,14 +10,19 @@
namespace DB
{
using Poco::SharedPtr;
class IBlockInputStream;
using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>;
using BlockInputStreams = std::vector<BlockInputStreamPtr>;
/** Коллбэк для отслеживания прогресса выполнения запроса.
* Используется в IProfilingBlockInputStream и Context-е.
* Функция принимает количество строк в последнем блоке, количество байт в последнем блоке.
* Следует иметь ввиду, что колбэк может вызываться из разных потоков.
*/
typedef std::function<void(const Progress & progress)> ProgressCallback;
using ProgressCallback = std::function<void(const Progress & progress)>;
/** Интерфейс потока для чтения данных по блокам из БД.
@ -28,9 +31,6 @@ typedef std::function<void(const Progress & progress)> ProgressCallback;
class IBlockInputStream : private boost::noncopyable
{
public:
typedef SharedPtr<IBlockInputStream> BlockInputStreamPtr;
typedef std::vector<BlockInputStreamPtr> BlockInputStreams;
IBlockInputStream() {}
/** Прочитать следующий блок.

View File

@ -2,8 +2,6 @@
#include <boost/noncopyable.hpp>
#include <Poco/SharedPtr.h>
#include <DB/Core/Block.h>
#include <DB/Core/Row.h>
#include <DB/Storages/IStorage.h>
@ -32,7 +30,7 @@ public:
/** Сбросить имеющиеся буферы для записи.
*/
virtual void flush() {}
/** Методы для установки дополнительной информации для вывода в поддерживающих её форматах.
*/
virtual void setRowsBeforeLimit(size_t rows_before_limit) {}
@ -48,9 +46,11 @@ public:
/** Не давать изменить таблицу, пока жив поток блоков.
*/
void addTableLock(const IStorage::TableStructureReadLockPtr & lock) { table_locks.push_back(lock); }
protected:
IStorage::TableStructureReadLocks table_locks;
};
using BlockOutputStreamPtr = std::shared_ptr<IBlockOutputStream>;
}

View File

@ -14,6 +14,9 @@ namespace DB
class QuotaForIntervals;
struct ProcessListElement;
class IProfilingBlockInputStream;
using ProfilingBlockInputStreamPtr = std::shared_ptr<IProfilingBlockInputStream>;
/** Смотрит за тем, как работает источник блоков.
@ -212,6 +215,4 @@ protected:
void collectAndSendTotalRowsApprox();
};
typedef SharedPtr<IProfilingBlockInputStream> ProfilingBlockInputStreamPtr;
}

View File

@ -1,7 +1,7 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <Poco/SharedPtr.h>
#include <memory>
namespace DB
@ -27,6 +27,6 @@ public:
virtual ~IRowInputStream() {}
};
using RowInputStreamPtr = Poco::SharedPtr<IRowInputStream>;
using RowInputStreamPtr = std::shared_ptr<IRowInputStream>;
}

View File

@ -1,7 +1,7 @@
#pragma once
#include <boost/noncopyable.hpp>
#include <memory>
#include <DB/Core/Block.h>
@ -46,6 +46,6 @@ public:
virtual ~IRowOutputStream() {}
};
typedef SharedPtr<IRowOutputStream> RowOutputStreamPtr;
using RowOutputStreamPtr = std::shared_ptr<IRowOutputStream>;
}

View File

@ -1,7 +1,5 @@
#pragma once
#include <Poco/SharedPtr.h>
#include <DB/Core/Block.h>
#include <DB/IO/WriteBuffer.h>
#include <DB/IO/WriteBufferValidUTF8.h>

View File

@ -1,15 +1,11 @@
#pragma once
#include <Poco/SharedPtr.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
namespace DB
{
using Poco::SharedPtr;
/** Реализует реляционную операцию LIMIT.
*/

View File

@ -115,7 +115,7 @@ private:
BlockInputStreamPtr block_in;
TemporaryFileStream(const std::string & path)
: file_in(path), compressed_in(file_in), block_in(new NativeBlockInputStream(compressed_in)) {}
: file_in(path), compressed_in(file_in), block_in(std::make_shared<NativeBlockInputStream>(compressed_in)) {}
};
std::vector<std::unique_ptr<TemporaryFileStream>> temporary_inputs;

View File

@ -7,8 +7,6 @@
namespace DB
{
using Poco::SharedPtr;
/** Доагрегирует поток блоков, в котором каждый блок уже агрегирован.
* Агрегатные функции в блоках не должны быть финализированы, чтобы их состояния можно было объединить.

View File

@ -111,7 +111,7 @@ private:
std::vector<Input> inputs;
using BlocksToMerge = Poco::SharedPtr<BlocksList>;
using BlocksToMerge = std::unique_ptr<BlocksList>;
void start();

View File

@ -146,13 +146,13 @@ protected:
size_t num_columns = 0;
std::vector<SharedBlockPtr> source_blocks;
typedef std::vector<SortCursorImpl> CursorImpls;
using CursorImpls = std::vector<SortCursorImpl>;
CursorImpls cursors;
typedef std::priority_queue<SortCursor> Queue;
using Queue = std::priority_queue<SortCursor>;
Queue queue;
typedef std::priority_queue<SortCursorWithCollation> QueueWithCollation;
using QueueWithCollation = std::priority_queue<SortCursorWithCollation>;
QueueWithCollation queue_with_collation;

View File

@ -14,8 +14,6 @@
namespace DB
{
using Poco::SharedPtr;
/** Агрегирует несколько источников параллельно.
* Производит агрегацию блоков из разных источников независимо в разных потоках, затем объединяет результаты.
@ -107,7 +105,7 @@ protected:
BlockInputStreams input_streams;
for (const auto & file : files.files)
{
temporary_inputs.emplace_back(new TemporaryFileStream(file->path()));
temporary_inputs.emplace_back(std::make_unique<TemporaryFileStream>(file->path()));
input_streams.emplace_back(temporary_inputs.back()->block_in);
}
@ -115,8 +113,8 @@ protected:
<< (files.sum_size_compressed / 1048576.0) << " MiB compressed, "
<< (files.sum_size_uncompressed / 1048576.0) << " MiB uncompressed.");
impl.reset(new MergingAggregatedMemoryEfficientBlockInputStream(
input_streams, params, final, temporary_data_merge_threads, temporary_data_merge_threads));
impl = std::make_unique<MergingAggregatedMemoryEfficientBlockInputStream>(
input_streams, params, final, temporary_data_merge_threads, temporary_data_merge_threads);
}
executed = true;
@ -156,7 +154,7 @@ private:
BlockInputStreamPtr block_in;
TemporaryFileStream(const std::string & path)
: file_in(path), compressed_in(file_in), block_in(new NativeBlockInputStream(compressed_in, ClickHouseRevision::get())) {}
: file_in(path), compressed_in(file_in), block_in(std::make_shared<NativeBlockInputStream>(compressed_in, ClickHouseRevision::get())) {}
};
std::vector<std::unique_ptr<TemporaryFileStream>> temporary_inputs;
@ -264,7 +262,7 @@ private:
Stopwatch watch;
for (auto & elem : many_data)
elem = new AggregatedDataVariants;
elem = std::make_shared<AggregatedDataVariants>();
processor.process();
processor.wait();

View File

@ -275,7 +275,7 @@ private:
Handler & handler;
/// Потоки.
typedef std::vector<std::thread> ThreadsData;
using ThreadsData = std::vector<std::thread>;
ThreadsData threads;
/** Набор доступных источников, которые не заняты каким-либо потоком в данный момент.
@ -295,7 +295,7 @@ private:
*
* Поэтому, используется очередь. Это можно улучшить в дальнейшем.
*/
typedef std::queue<InputData> AvailableInputs;
using AvailableInputs = std::queue<InputData>;
AvailableInputs available_inputs;
/// Для операций с available_inputs.

View File

@ -28,7 +28,7 @@ protected:
void writeTotals();
void writeExtremes();
typedef std::vector<size_t> Widths_t;
using Widths_t = std::vector<size_t>;
/// Вычислить видимую (при выводе на консоль с кодировкой UTF-8) ширину значений и имён столбцов.
void calculateWidths(Block & block, Widths_t & max_widths, Widths_t & name_widths);

View File

@ -19,7 +19,7 @@ public:
void writeSuffix() override;
private:
typedef std::vector<Block> Blocks_t;
using Blocks_t = std::vector<Block>;
Blocks_t blocks;
};

View File

@ -32,7 +32,7 @@ public:
Dependencies dependencies = context.getDependencies(database, table);
for (size_t i = 0; i < dependencies.size(); ++i)
{
children.push_back(new PushingToViewsBlockOutputStream(dependencies[i].first, dependencies[i].second, context, ASTPtr()));
children.push_back(std::make_shared<PushingToViewsBlockOutputStream>(dependencies[i].first, dependencies[i].second, context, ASTPtr()));
queries.push_back(dynamic_cast<StorageView &>(*context.getTable(dependencies[i].first, dependencies[i].second)).getInnerQuery());
}
@ -44,9 +44,9 @@ public:
{
for (size_t i = 0; i < children.size(); ++i)
{
BlockInputStreamPtr from = new OneBlockInputStream(block);
BlockInputStreamPtr from = std::make_shared<OneBlockInputStream>(block);
InterpreterSelectQuery select(queries[i], context, QueryProcessingStage::Complete, 0, from);
BlockInputStreamPtr data = new MaterializingBlockInputStream(select.execute().in);
BlockInputStreamPtr data = std::make_shared<MaterializingBlockInputStream>(select.execute().in);
copyData(*data, *children[i]);
}

View File

@ -62,7 +62,7 @@ protected:
private:
size_t queue_size;
typedef ConcurrentBoundedQueue<Block> Queue;
using Queue = ConcurrentBoundedQueue<Block>;
Queue queue;
};

View File

@ -1,7 +1,5 @@
#pragma once
#include <Poco/SharedPtr.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/Columns/ColumnConst.h>
@ -10,7 +8,6 @@
namespace DB
{
/** Удаляет из блока указанные столбцы.
*/
class RemoveColumnsBlockInputStream : public IProfilingBlockInputStream

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