Using std::shared_ptr for data types [#METR-21503].

This commit is contained in:
Alexey Milovidov 2016-05-28 17:14:18 +03:00
parent 3c1c9d46f7
commit 646d70a545
91 changed files with 513 additions and 689 deletions

View File

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

View File

@ -8,9 +8,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
/** Интерфейс для пулов соединений. /** Интерфейс для пулов соединений.
* *
* Использование (на примере обычного ConnectionPool): * Использование (на примере обычного ConnectionPool):

View File

@ -2,8 +2,6 @@
#include <string.h> // memcpy #include <string.h> // memcpy
#include <Poco/SharedPtr.h>
#include <DB/Common/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Common/Arena.h> #include <DB/Common/Arena.h>
@ -22,8 +20,6 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
} }
using Poco::SharedPtr;
/** Cтолбeц значений типа массив. /** Cтолбeц значений типа массив.
* В памяти он представлен, как один столбец вложенного типа, размер которого равен сумме размеров всех массивов, * В памяти он представлен, как один столбец вложенного типа, размер которого равен сумме размеров всех массивов,
* и как массив смещений в нём, который позволяет достать каждый элемент. * и как массив смещений в нём, который позволяет достать каждый элемент.

View File

@ -1,7 +1,5 @@
#pragma once #pragma once
#include <Poco/SharedPtr.h>
#include <DB/Core/Field.h> #include <DB/Core/Field.h>
#include <DB/Common/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Columns/ColumnVector.h> #include <DB/Columns/ColumnVector.h>
@ -21,8 +19,6 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
using Poco::SharedPtr;
class IColumnConst : public IColumn class IColumnConst : public IColumn
{ {
@ -57,11 +53,11 @@ namespace ColumnConstDetails
/** Столбец-константа может содержать внутри себя само значение, /** Столбец-константа может содержать внутри себя само значение,
* или, в случае массивов, SharedPtr от значения-массива, * или, в случае массивов, std::shared_ptr от значения-массива,
* чтобы избежать проблем производительности при копировании очень больших массивов. * чтобы избежать проблем производительности при копировании очень больших массивов.
* *
* T - тип значения, * T - тип значения,
* DataHolder - как значение хранится в таблице (либо T, либо SharedPtr<T>) * DataHolder - как значение хранится в таблице (либо T, либо std::shared_ptr<T>)
* Derived должен реализовать методы getDataFromHolderImpl - получить ссылку на значение из holder-а. * Derived должен реализовать методы getDataFromHolderImpl - получить ссылку на значение из holder-а.
* *
* Для строк и массивов реализации sizeOfField и byteSize могут быть некорректными. * Для строк и массивов реализации sizeOfField и byteSize могут быть некорректными.
@ -232,10 +228,10 @@ public:
template <> 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: private:
friend class ColumnConstBase<Array, SharedPtr<Array>, ColumnConst<Array>>; friend class ColumnConstBase<Array, std::shared_ptr<Array>, ColumnConst<Array>>;
Array & getDataFromHolderImpl() { return *data; } Array & getDataFromHolderImpl() { return *data; }
const Array & getDataFromHolderImpl() const { return *data; } const Array & getDataFromHolderImpl() const { return *data; }
@ -243,10 +239,10 @@ private:
public: public:
/// data_type_ должен быть ненулевым. /// data_type_ должен быть ненулевым.
ColumnConst(size_t s_, const Array & data_, DataTypePtr data_type_ = DataTypePtr()) 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()) ColumnConst(size_t s_, const std::shared_ptr<Array> & data_, DataTypePtr data_type_ = DataTypePtr())
: ColumnConstBase<Array, SharedPtr<Array>, ColumnConst<Array>>(s_, data_, data_type_) {} : ColumnConstBase<Array, std::shared_ptr<Array>, ColumnConst<Array>>(s_, data_, data_type_) {}
StringRef getDataAt(size_t n) const override; StringRef getDataAt(size_t n) const override;
StringRef getDataAtWithTerminatingZero(size_t n) const override; StringRef getDataAtWithTerminatingZero(size_t n) const override;
@ -267,10 +263,10 @@ public:
template <> 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: private:
friend class ColumnConstBase<Tuple, SharedPtr<Tuple>, ColumnConst<Tuple>>; friend class ColumnConstBase<Tuple, std::shared_ptr<Tuple>, ColumnConst<Tuple>>;
Tuple & getDataFromHolderImpl() { return *data; } Tuple & getDataFromHolderImpl() { return *data; }
const Tuple & getDataFromHolderImpl() const { return *data; } const Tuple & getDataFromHolderImpl() const { return *data; }
@ -278,10 +274,10 @@ private:
public: public:
/// data_type_ должен быть ненулевым. /// data_type_ должен быть ненулевым.
ColumnConst(size_t s_, const Tuple & data_, DataTypePtr data_type_ = DataTypePtr()) 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()) ColumnConst(size_t s_, const std::shared_ptr<Tuple> & data_, DataTypePtr data_type_ = DataTypePtr())
: ColumnConstBase<Tuple, SharedPtr<Tuple>, ColumnConst<Tuple>>(s_, data_, data_type_) {} : ColumnConstBase<Tuple, std::shared_ptr<Tuple>, ColumnConst<Tuple>>(s_, data_, data_type_) {}
StringRef getDataAt(size_t n) const override; StringRef getDataAt(size_t n) const override;
StringRef getDataAtWithTerminatingZero(size_t n) const override; StringRef getDataAtWithTerminatingZero(size_t n) const override;

View File

@ -2,9 +2,9 @@
#include <cerrno> #include <cerrno>
#include <vector> #include <vector>
#include <memory>
#include <Poco/Exception.h> #include <Poco/Exception.h>
#include <Poco/SharedPtr.h>
#include <DB/Common/StackTrace.h> #include <DB/Common/StackTrace.h>
@ -90,7 +90,7 @@ std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace);
void rethrowFirstException(Exceptions & exceptions); void rethrowFirstException(Exceptions & exceptions);
Poco::SharedPtr<Poco::Exception> convertCurrentException(); std::unique_ptr<Poco::Exception> convertCurrentException();
template <typename T> template <typename T>

View File

@ -4,8 +4,6 @@
#include <vector> #include <vector>
#include <memory> #include <memory>
#include <Poco/SharedPtr.h>
#include <re2/re2.h> #include <re2/re2.h>
#include <re2_st/re2.h> #include <re2_st/re2.h>

View File

@ -84,7 +84,7 @@ public:
Object & operator*() & { return *data->data.object; } Object & operator*() & { return *data->data.object; }
const Object & operator*() const & { 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: private:
std::shared_ptr<PoolEntryHelper> data; std::shared_ptr<PoolEntryHelper> data;

View File

@ -1,6 +1,7 @@
#pragma once #pragma once
#include <vector> #include <vector>
#include <algorithm>
#include <type_traits> #include <type_traits>
#include <functional> #include <functional>
@ -27,8 +28,6 @@ using TupleBackend = std::vector<Field>;
STRONG_TYPEDEF(TupleBackend, Tuple); /// Значение типа "кортеж" STRONG_TYPEDEF(TupleBackend, Tuple); /// Значение типа "кортеж"
using Poco::SharedPtr;
/** 32 хватает с запасом (достаточно 28), но выбрано круглое число, /** 32 хватает с запасом (достаточно 28), но выбрано круглое число,
* чтобы арифметика при использовании массивов из Field была проще (не содержала умножения). * чтобы арифметика при использовании массивов из Field была проще (не содержала умножения).
*/ */

View File

@ -18,12 +18,12 @@ struct SortColumnDescription
String column_name; /// Имя столбца. String column_name; /// Имя столбца.
size_t column_number; /// Номер столбца (используется, если не задано имя). size_t column_number; /// Номер столбца (используется, если не задано имя).
int direction; /// 1 - по возрастанию, -1 - по убыванию. 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_) {} : 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_) {} : column_name(column_name_), column_number(0), direction(direction_), collator(collator_) {}
/// Для IBlockInputStream. /// Для IBlockInputStream.
@ -31,7 +31,7 @@ struct SortColumnDescription
{ {
std::stringstream res; std::stringstream res;
res << column_name << ", " << column_number << ", " << direction; res << column_name << ", " << column_number << ", " << direction;
if (!collator.isNull()) if (collator)
res << ", collation locale: " << collator->getLocale(); res << ", collation locale: " << collator->getLocale();
return res.str(); return res.str();
} }
@ -96,7 +96,7 @@ struct SortCursorImpl
sort_columns.push_back(&*block.getByPosition(column_number).column); 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]; has_collation |= need_collation[j];
} }

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -14,8 +14,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
/** Агрегирует несколько источников параллельно. /** Агрегирует несколько источников параллельно.
* Производит агрегацию блоков из разных источников независимо в разных потоках, затем объединяет результаты. * Производит агрегацию блоков из разных источников независимо в разных потоках, затем объединяет результаты.
@ -264,7 +262,7 @@ private:
Stopwatch watch; Stopwatch watch;
for (auto & elem : many_data) for (auto & elem : many_data)
elem = new AggregatedDataVariants; elem = std::make_shared<AggregatedDataVariants>();
processor.process(); processor.process();
processor.wait(); processor.wait();

View File

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

View File

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

View File

@ -10,8 +10,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;

View File

@ -1,7 +1,5 @@
#pragma once #pragma once
#include <Poco/SharedPtr.h>
#include <DB/Core/Block.h> #include <DB/Core/Block.h>
#include <DB/Core/Names.h> #include <DB/Core/Names.h>
#include <DB/IO/WriteBuffer.h> #include <DB/IO/WriteBuffer.h>
@ -11,9 +9,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
/** Поток для вывода данных в формате "каждое значение на своей строке". /** Поток для вывода данных в формате "каждое значение на своей строке".
*/ */
class VerticalRowOutputStream : public IRowOutputStream class VerticalRowOutputStream : public IRowOutputStream

View File

@ -7,8 +7,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
class DataTypeArray final : public IDataType class DataTypeArray final : public IDataType
{ {

View File

@ -2,17 +2,12 @@
#include <ostream> #include <ostream>
#include <Poco/SharedPtr.h>
#include <DB/DataTypes/IDataType.h> #include <DB/DataTypes/IDataType.h>
namespace DB namespace DB
{ {
using Poco::SharedPtr;
class DataTypeString final : public IDataType class DataTypeString final : public IDataType
{ {
public: public:

View File

@ -185,10 +185,10 @@ public:
using base_type = FunctionTransformWithDictionary; using base_type = FunctionTransformWithDictionary;
private: private:
const SharedPtr<typename DictGetter::Src> owned_dict; const std::shared_ptr<typename DictGetter::Src> owned_dict;
public: public:
FunctionTransformWithDictionary(const SharedPtr<typename DictGetter::Src> & owned_dict_) FunctionTransformWithDictionary(const std::shared_ptr<typename DictGetter::Src> & owned_dict_)
: owned_dict(owned_dict_) : owned_dict(owned_dict_)
{ {
if (!owned_dict) if (!owned_dict)
@ -278,10 +278,10 @@ public:
using base_type = FunctionIsInWithDictionary; using base_type = FunctionIsInWithDictionary;
private: private:
const SharedPtr<typename DictGetter::Src> owned_dict; const std::shared_ptr<typename DictGetter::Src> owned_dict;
public: public:
FunctionIsInWithDictionary(const SharedPtr<typename DictGetter::Src> & owned_dict_) FunctionIsInWithDictionary(const std::shared_ptr<typename DictGetter::Src> & owned_dict_)
: owned_dict(owned_dict_) : owned_dict(owned_dict_)
{ {
if (!owned_dict) if (!owned_dict)
@ -411,10 +411,10 @@ public:
using base_type = FunctionHierarchyWithDictionary; using base_type = FunctionHierarchyWithDictionary;
private: private:
const SharedPtr<typename DictGetter::Src> owned_dict; const std::shared_ptr<typename DictGetter::Src> owned_dict;
public: public:
FunctionHierarchyWithDictionary(const SharedPtr<typename DictGetter::Src> & owned_dict_) FunctionHierarchyWithDictionary(const std::shared_ptr<typename DictGetter::Src> & owned_dict_)
: owned_dict(owned_dict_) : owned_dict(owned_dict_)
{ {
if (!owned_dict) if (!owned_dict)
@ -684,10 +684,10 @@ public:
} }
private: private:
const SharedPtr<RegionsNames> owned_dict; const std::shared_ptr<RegionsNames> owned_dict;
public: public:
FunctionRegionToName(const SharedPtr<RegionsNames> & owned_dict_) FunctionRegionToName(const std::shared_ptr<RegionsNames> & owned_dict_)
: owned_dict(owned_dict_) : owned_dict(owned_dict_)
{ {
if (!owned_dict) if (!owned_dict)

View File

@ -4,7 +4,6 @@
#include <vector> #include <vector>
#include <Poco/SharedPtr.h>
#include <Poco/Net/NetException.h> #include <Poco/Net/NetException.h>
#include <threadpool.hpp> #include <threadpool.hpp>
@ -15,8 +14,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
/** Записывает данные асинхронно с помощью двойной буферизации. /** Записывает данные асинхронно с помощью двойной буферизации.
*/ */

View File

@ -3,7 +3,6 @@
#include <DB/IO/WriteBuffer.h> #include <DB/IO/WriteBuffer.h>
#include <DB/IO/HashingWriteBuffer.h> #include <DB/IO/HashingWriteBuffer.h>
#include <Poco/SharedPtr.h>
#include <Poco/Net/HTTPClientSession.h> #include <Poco/Net/HTTPClientSession.h>
namespace DB namespace DB

View File

@ -15,7 +15,7 @@ namespace DB
class RemoteReadBuffer : public ReadBuffer class RemoteReadBuffer : public ReadBuffer
{ {
private: private:
Poco::SharedPtr<ReadBufferFromHTTP> impl; std::unique_ptr<ReadBufferFromHTTP> impl;
public: public:
RemoteReadBuffer( RemoteReadBuffer(
@ -34,7 +34,7 @@ public:
std::make_pair("path", path), std::make_pair("path", path),
std::make_pair("compress", (compress ? "true" : "false"))}; std::make_pair("compress", (compress ? "true" : "false"))};
impl = new ReadBufferFromHTTP(host, port, params, buffer_size, connection_timeout, send_timeout, receive_timeout); impl = std::make_unique<ReadBufferFromHTTP>(host, port, params, buffer_size, connection_timeout, send_timeout, receive_timeout);
} }
bool nextImpl() bool nextImpl()

View File

@ -752,7 +752,7 @@ struct AggregatedDataVariants : private boost::noncopyable
M(serialized_two_level) M(serialized_two_level)
}; };
using AggregatedDataVariantsPtr = SharedPtr<AggregatedDataVariants>; using AggregatedDataVariantsPtr = std::shared_ptr<AggregatedDataVariants>;
using ManyAggregatedDataVariants = std::vector<AggregatedDataVariantsPtr>; using ManyAggregatedDataVariants = std::vector<AggregatedDataVariantsPtr>;
/** Как считаются "тотальные" значения при наличии WITH TOTALS? /** Как считаются "тотальные" значения при наличии WITH TOTALS?

View File

@ -278,7 +278,7 @@ public:
void resetCaches() const; void resetCaches() const;
const Cluster & getCluster(const std::string & cluster_name) const; const Cluster & getCluster(const std::string & cluster_name) const;
Poco::SharedPtr<Clusters> getClusters() const; std::shared_ptr<Clusters> getClusters() const;
Compiler & getCompiler(); Compiler & getCompiler();
QueryLog & getQueryLog(); QueryLog & getQueryLog();

View File

@ -1,5 +1,6 @@
#pragma once #pragma once
#include <Poco/SharedPtr.h>
#include <DB/Core/Field.h> #include <DB/Core/Field.h>
@ -13,6 +14,6 @@ class Context;
* Используется в редких случаях - для элемента множества в IN, для данных для INSERT. * Используется в редких случаях - для элемента множества в IN, для данных для INSERT.
* Весьма неоптимально. * Весьма неоптимально.
*/ */
Field evaluateConstantExpression(SharedPtr<IAST> & node, const Context & context); Field evaluateConstantExpression(Poco::SharedPtr<IAST> & node, const Context & context);
} }

View File

@ -6,9 +6,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
/** Список выражений типа "a, b + c, f(d)" /** Список выражений типа "a, b + c, f(d)"
*/ */
class ASTExpressionList : public IAST class ASTExpressionList : public IAST

View File

@ -6,9 +6,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
/** Элемент выражения, после которого стоит ASC или DESC /** Элемент выражения, после которого стоит ASC или DESC
*/ */
class ASTOrderByElement : public IAST class ASTOrderByElement : public IAST
@ -19,10 +16,10 @@ public:
/** Collator для locale-specific сортировки строк. /** Collator для locale-specific сортировки строк.
* Если nullptr, то производится сортировка по байтам. * Если nullptr, то производится сортировка по байтам.
*/ */
Poco::SharedPtr<Collator> collator; std::shared_ptr<Collator> collator;
ASTOrderByElement() = default; ASTOrderByElement() = default;
ASTOrderByElement(const StringRange range_, const int direction_, const Poco::SharedPtr<Collator> & collator_ = nullptr) ASTOrderByElement(const StringRange range_, const int direction_, const std::shared_ptr<Collator> & collator_ = nullptr)
: IAST(range_), direction(direction_), collator(collator_) {} : IAST(range_), direction(direction_), collator(collator_) {}
/** Получить текст, который идентифицирует этот элемент. */ /** Получить текст, который идентифицирует этот элемент. */
@ -34,8 +31,11 @@ protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{ {
children.front()->formatImpl(settings, state, frame); children.front()->formatImpl(settings, state, frame);
settings.ostr << (settings.hilite ? hilite_keyword : "") << (direction == -1 ? " DESC" : " ASC") << (settings.hilite ? hilite_none : ""); settings.ostr << (settings.hilite ? hilite_keyword : "")
if (!collator.isNull()) << (direction == -1 ? " DESC" : " ASC")
<< (settings.hilite ? hilite_none : "");
if (collator)
{ {
settings.ostr << (settings.hilite ? hilite_keyword : "") << " COLLATE " << (settings.hilite ? hilite_none : "") settings.ostr << (settings.hilite ? hilite_keyword : "") << " COLLATE " << (settings.hilite ? hilite_none : "")
<< "'" << collator->getLocale() << "'"; << "'" << collator->getLocale() << "'";

View File

@ -8,8 +8,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
/** Если прямо сейчас не s, то ошибка. /** Если прямо сейчас не s, то ошибка.
* Если word_boundary установлен в true, и последний символ строки - словарный (\w), * Если word_boundary установлен в true, и последний символ строки - словарный (\w),

View File

@ -11,8 +11,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
using Expected = const char *; using Expected = const char *;

View File

@ -8,7 +8,6 @@
#include <mutex> #include <mutex>
#include <Poco/RWLock.h> #include <Poco/RWLock.h>
#include <Poco/Event.h> #include <Poco/Event.h>
#include <Poco/SharedPtr.h>
#include <DB/Core/Types.h> #include <DB/Core/Types.h>
namespace DB namespace DB

View File

@ -368,8 +368,9 @@ private:
if (storage.merging_params.mode != MergeTreeData::MergingParams::Unsorted) if (storage.merging_params.mode != MergeTreeData::MergingParams::Unsorted)
{ {
index_file_stream = new WriteBufferFromFile(part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY); index_file_stream = std::make_unique<WriteBufferFromFile>(
index_stream = new HashingWriteBuffer(*index_file_stream); part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY);
index_stream = std::make_unique<HashingWriteBuffer>(*index_file_stream);
} }
} }
@ -467,12 +468,11 @@ private:
size_t marks_count = 0; size_t marks_count = 0;
SharedPtr<WriteBufferFromFile> index_file_stream; std::unique_ptr<WriteBufferFromFile> index_file_stream;
SharedPtr<HashingWriteBuffer> index_stream; std::unique_ptr<HashingWriteBuffer> index_stream;
MergeTreeData::DataPart::Index index_columns; MergeTreeData::DataPart::Index index_columns;
}; };
using MergedBlockOutputStreamPtr = Poco::SharedPtr<MergedBlockOutputStream>;
/// Записывает только те, столбцы, что лежат в block /// Записывает только те, столбцы, что лежат в block
class MergedColumnOnlyOutputStream : public IMergedBlockOutputStream class MergedColumnOnlyOutputStream : public IMergedBlockOutputStream

View File

@ -8,7 +8,6 @@
#include <zkutil/SingleBarrier.h> #include <zkutil/SingleBarrier.h>
#include <Poco/Util/LayeredConfiguration.h> #include <Poco/Util/LayeredConfiguration.h>
#include <Poco/SharedPtr.h>
#include <string> #include <string>
#include <thread> #include <thread>

View File

@ -42,7 +42,7 @@ public:
NamesAndTypesListPtr columns_, /// Список столбцов. NamesAndTypesListPtr columns_, /// Список столбцов.
const String & remote_database_, /// БД на удалённых серверах. const String & remote_database_, /// БД на удалённых серверах.
const String & remote_table_, /// Имя таблицы на удалённых серверах. const String & remote_table_, /// Имя таблицы на удалённых серверах.
SharedPtr<Cluster> & owned_cluster_, std::shared_ptr<Cluster> & owned_cluster_,
Context & context_); Context & context_);
std::string getName() const override { return "Distributed"; } std::string getName() const override { return "Distributed"; }
@ -134,8 +134,9 @@ private:
Context & context; Context & context;
Logger * log = &Logger::get("StorageDistributed"); Logger * log = &Logger::get("StorageDistributed");
/// Используется только, если таблица должна владеть объектом Cluster, которым больше никто не владеет - для реализации TableFunctionRemote. /// Используется только, если таблица должна владеть объектом Cluster,
SharedPtr<Cluster> owned_cluster; /// которым больше никто не владеет - для реализации TableFunctionRemote.
std::shared_ptr<Cluster> owned_cluster;
/// Соединения с удалёнными серверами. /// Соединения с удалёнными серверами.
const Cluster & cluster; const Cluster & cluster;

View File

@ -1,7 +1,5 @@
#pragma once #pragma once
#include <Poco/SharedPtr.h>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h> #include <DB/Interpreters/Context.h>
@ -9,9 +7,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
/** Реализует системную таблицу databases, которая позволяет получить информацию о всех БД. /** Реализует системную таблицу databases, которая позволяет получить информацию о всех БД.
*/ */
class StorageSystemDatabases : public IStorage class StorageSystemDatabases : public IStorage

View File

@ -1,16 +1,11 @@
#pragma once #pragma once
#include <Poco/SharedPtr.h>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
namespace DB namespace DB
{ {
using Poco::SharedPtr;
/** Реализует хранилище для системной таблицы Numbers. /** Реализует хранилище для системной таблицы Numbers.
* Таблица содержит единственный столбец number UInt64. * Таблица содержит единственный столбец number UInt64.
* Из этой таблицы можно прочитать все натуральные числа, начиная с 0 (до 2^64 - 1, а потом заново). * Из этой таблицы можно прочитать все натуральные числа, начиная с 0 (до 2^64 - 1, а потом заново).

View File

@ -1,7 +1,5 @@
#pragma once #pragma once
#include <Poco/SharedPtr.h>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h> #include <DB/Interpreters/Context.h>
@ -9,9 +7,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
/** Реализует системную таблицу tables, которая позволяет получить информацию о всех таблицах. /** Реализует системную таблицу tables, которая позволяет получить информацию о всех таблицах.
*/ */
class StorageSystemParts : public IStorage class StorageSystemParts : public IStorage

View File

@ -1,7 +1,5 @@
#pragma once #pragma once
#include <Poco/SharedPtr.h>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h> #include <DB/Interpreters/Context.h>
@ -9,9 +7,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
/** Реализует системную таблицу processes, которая позволяет получить информацию о запросах, исполняющихся в данный момент. /** Реализует системную таблицу processes, которая позволяет получить информацию о запросах, исполняющихся в данный момент.
*/ */
class StorageSystemProcesses : public IStorage class StorageSystemProcesses : public IStorage

View File

@ -1,7 +1,5 @@
#pragma once #pragma once
#include <Poco/SharedPtr.h>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h> #include <DB/Interpreters/Context.h>
@ -9,9 +7,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
/** Реализует системную таблицу replicas, которая позволяет получить информацию о статусе реплицируемых таблиц. /** Реализует системную таблицу replicas, которая позволяет получить информацию о статусе реплицируемых таблиц.
*/ */
class StorageSystemReplicas : public IStorage class StorageSystemReplicas : public IStorage

View File

@ -1,7 +1,5 @@
#pragma once #pragma once
#include <Poco/SharedPtr.h>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h> #include <DB/Interpreters/Context.h>
@ -9,9 +7,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
/** Реализует системную таблицу replication_queue, которая позволяет посмотреть очереди репликации для реплицируемых таблиц. /** Реализует системную таблицу replication_queue, которая позволяет посмотреть очереди репликации для реплицируемых таблиц.
*/ */
class StorageSystemReplicationQueue : public IStorage class StorageSystemReplicationQueue : public IStorage

View File

@ -1,7 +1,5 @@
#pragma once #pragma once
#include <Poco/SharedPtr.h>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h> #include <DB/Interpreters/Context.h>
@ -9,9 +7,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
/** Реализует системную таблицу settings, которая позволяет получить информацию о текущих настройках. /** Реализует системную таблицу settings, которая позволяет получить информацию о текущих настройках.
*/ */
class StorageSystemSettings : public IStorage class StorageSystemSettings : public IStorage

View File

@ -1,7 +1,5 @@
#pragma once #pragma once
#include <Poco/SharedPtr.h>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h> #include <DB/Interpreters/Context.h>
@ -9,9 +7,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
/** Реализует системную таблицу tables, которая позволяет получить информацию о всех таблицах. /** Реализует системную таблицу tables, которая позволяет получить информацию о всех таблицах.
*/ */
class StorageSystemTables : public IStorage class StorageSystemTables : public IStorage

View File

@ -1,7 +1,5 @@
#pragma once #pragma once
#include <Poco/SharedPtr.h>
#include <DB/Storages/IStorage.h> #include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h> #include <DB/Interpreters/Context.h>
@ -9,9 +7,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
/** Реализует системную таблицу zookeeper, которая позволяет просматривать данные в ZooKeeper в целях отладки. /** Реализует системную таблицу zookeeper, которая позволяет просматривать данные в ZooKeeper в целях отладки.
*/ */
class StorageSystemZooKeeper : public IStorage class StorageSystemZooKeeper : public IStorage

View File

@ -10,7 +10,6 @@
#include <random> #include <random>
#include <Poco/File.h> #include <Poco/File.h>
#include <Poco/SharedPtr.h>
#include <Poco/Util/Application.h> #include <Poco/Util/Application.h>
#include <DB/Common/Stopwatch.h> #include <DB/Common/Stopwatch.h>

View File

@ -13,7 +13,6 @@
#include <boost/program_options.hpp> #include <boost/program_options.hpp>
#include <Poco/File.h> #include <Poco/File.h>
#include <Poco/SharedPtr.h>
#include <Poco/Util/Application.h> #include <Poco/Util/Application.h>
#include <common/ClickHouseRevision.h> #include <common/ClickHouseRevision.h>
@ -114,8 +113,6 @@ namespace ErrorCodes
extern const int CLIENT_OUTPUT_FORMAT_SPECIFIED; extern const int CLIENT_OUTPUT_FORMAT_SPECIFIED;
} }
using Poco::SharedPtr;
class Client : public Poco::Util::Application class Client : public Poco::Util::Application
{ {
@ -172,7 +169,7 @@ private:
ASTPtr parsed_query; ASTPtr parsed_query;
/// Последнее полученное от сервера исключение. Для кода возврата в неинтерактивном режиме. /// Последнее полученное от сервера исключение. Для кода возврата в неинтерактивном режиме.
Poco::SharedPtr<DB::Exception> last_exception; std::unique_ptr<Exception> last_exception;
/// Было ли в последнем запросе исключение. /// Было ли в последнем запросе исключение.
bool got_exception = false; bool got_exception = false;
@ -881,7 +878,7 @@ private:
case Protocol::Server::Exception: case Protocol::Server::Exception:
onException(*packet.exception); onException(*packet.exception);
last_exception = packet.exception; last_exception = std::move(packet.exception);
return false; return false;
case Protocol::Server::EndOfStream: case Protocol::Server::EndOfStream:
@ -908,7 +905,7 @@ private:
case Protocol::Server::Exception: case Protocol::Server::Exception:
onException(*packet.exception); onException(*packet.exception);
last_exception = packet.exception; last_exception = std::move(packet.exception);
return false; return false;
default: default:

View File

@ -51,8 +51,8 @@ void Connection::connect()
socket.setSendTimeout(send_timeout); socket.setSendTimeout(send_timeout);
socket.setNoDelay(true); socket.setNoDelay(true);
in = new ReadBufferFromPocoSocket(socket); in = std::make_shared<ReadBufferFromPocoSocket>(socket);
out = new WriteBufferFromPocoSocket(socket); out = std::make_shared<WriteBufferFromPocoSocket>(socket);
connected = true; connected = true;
@ -275,10 +275,10 @@ void Connection::sendQuery(const String & query, const String & query_id_, UInt6
writeStringBinary(query, *out); writeStringBinary(query, *out);
maybe_compressed_in = nullptr; maybe_compressed_in.reset();
maybe_compressed_out = nullptr; maybe_compressed_out.reset();
block_in = nullptr; block_in.reset();
block_out = nullptr; block_out.reset();
/// Если версия сервера достаточно новая и стоит флаг, отправляем пустой блок, символизируя конец передачи данных. /// Если версия сервера достаточно новая и стоит флаг, отправляем пустой блок, символизируя конец передачи данных.
if (server_revision >= DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES && !with_pending_data) if (server_revision >= DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES && !with_pending_data)
@ -305,7 +305,7 @@ void Connection::sendData(const Block & block, const String & name)
if (!block_out) if (!block_out)
{ {
if (compression == Protocol::Compression::Enable) if (compression == Protocol::Compression::Enable)
maybe_compressed_out = new CompressedWriteBuffer(*out, network_compression_method); maybe_compressed_out = std::make_shared<CompressedWriteBuffer>(*out, network_compression_method);
else else
maybe_compressed_out = out; maybe_compressed_out = out;
@ -502,7 +502,7 @@ void Connection::initBlockInput()
if (!block_in) if (!block_in)
{ {
if (compression == Protocol::Compression::Enable) if (compression == Protocol::Compression::Enable)
maybe_compressed_in = new CompressedReadBuffer(*in); maybe_compressed_in = std::make_shared<CompressedReadBuffer>(*in);
else else
maybe_compressed_in = in; maybe_compressed_in = in;
@ -521,13 +521,13 @@ void Connection::setDescription()
} }
SharedPtr<Exception> Connection::receiveException() std::unique_ptr<Exception> Connection::receiveException()
{ {
//LOG_TRACE(log_wrapper.get(), "Receiving exception"); //LOG_TRACE(log_wrapper.get(), "Receiving exception");
Exception e; Exception e;
readException(e, *in, "Received from " + getDescription()); readException(e, *in, "Received from " + getDescription());
return e.clone(); return { e.clone() };
} }

View File

@ -114,7 +114,7 @@ std::string getCurrentExceptionMessage(bool with_stacktrace)
} }
Poco::SharedPtr<Poco::Exception> convertCurrentException() std::unique_ptr<Poco::Exception> convertCurrentException()
{ {
try try
{ {
@ -122,19 +122,19 @@ Poco::SharedPtr<Poco::Exception> convertCurrentException()
} }
catch (const Exception & e) catch (const Exception & e)
{ {
return e.clone(); return std::unique_ptr<Poco::Exception>{ e.clone() };
} }
catch (const Poco::Exception & e) catch (const Poco::Exception & e)
{ {
return e.clone(); return std::unique_ptr<Poco::Exception>{ e.clone() };
} }
catch (const std::exception & e) catch (const std::exception & e)
{ {
return new Exception(e.what(), ErrorCodes::STD_EXCEPTION); return std::make_unique<Exception>(e.what(), ErrorCodes::STD_EXCEPTION);
} }
catch (...) catch (...)
{ {
return new Exception("Unknown exception", ErrorCodes::UNKNOWN_EXCEPTION); return std::make_unique<Exception>("Unknown exception", ErrorCodes::UNKNOWN_EXCEPTION);
} }
} }

View File

@ -12,7 +12,7 @@ Block AggregatingBlockInputStream::readImpl()
if (!executed) if (!executed)
{ {
executed = true; executed = true;
AggregatedDataVariantsPtr data_variants = new AggregatedDataVariants; AggregatedDataVariantsPtr data_variants = std::make_shared<AggregatedDataVariants>();
Aggregator::CancellationHook hook = [&]() { return this->isCancelled(); }; Aggregator::CancellationHook hook = [&]() { return this->isCancelled(); };
aggregator.setCancellationHook(hook); aggregator.setCancellationHook(hook);

View File

@ -6,9 +6,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
JSONCompactRowOutputStream::JSONCompactRowOutputStream(WriteBuffer & ostr_, const Block & sample_) JSONCompactRowOutputStream::JSONCompactRowOutputStream(WriteBuffer & ostr_, const Block & sample_)
: JSONRowOutputStream(ostr_, sample_) : JSONRowOutputStream(ostr_, sample_)
{ {

View File

@ -6,8 +6,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
LimitBlockInputStream::LimitBlockInputStream(BlockInputStreamPtr input_, size_t limit_, size_t offset_, bool always_read_till_end_) LimitBlockInputStream::LimitBlockInputStream(BlockInputStreamPtr input_, size_t limit_, size_t offset_, bool always_read_till_end_)
: limit(limit_), offset(offset_), always_read_till_end(always_read_till_end_) : limit(limit_), offset(offset_), always_read_till_end(always_read_till_end_)
{ {

View File

@ -6,9 +6,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
TabSeparatedRowOutputStream::TabSeparatedRowOutputStream(WriteBuffer & ostr_, const Block & sample_, bool with_names_, bool with_types_) TabSeparatedRowOutputStream::TabSeparatedRowOutputStream(WriteBuffer & ostr_, const Block & sample_, bool with_names_, bool with_types_)
: ostr(ostr_), sample(sample_), with_names(with_names_), with_types(with_types_) : ostr(ostr_), sample(sample_), with_names(with_names_), with_types(with_types_)
{ {

View File

@ -7,9 +7,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
VerticalRowOutputStream::VerticalRowOutputStream(WriteBuffer & ostr_, const Block & sample_) VerticalRowOutputStream::VerticalRowOutputStream(WriteBuffer & ostr_, const Block & sample_)
: ostr(ostr_), sample(sample_), field_number(0), row_number(0) : ostr(ostr_), sample(sample_), field_number(0), row_number(0)
{ {

View File

@ -6,9 +6,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
XMLRowOutputStream::XMLRowOutputStream(WriteBuffer & ostr_, const Block & sample_) XMLRowOutputStream::XMLRowOutputStream(WriteBuffer & ostr_, const Block & sample_)
: dst_ostr(ostr_) : dst_ostr(ostr_)
{ {

View File

@ -15,81 +15,76 @@
#include <DB/DataTypes/DataTypesNumberFixed.h> #include <DB/DataTypes/DataTypesNumberFixed.h>
using Poco::SharedPtr;
int main(int argc, char ** argv) int main(int argc, char ** argv)
try
{ {
using namespace DB; using namespace DB;
try Poco::AutoPtr<Poco::ConsoleChannel> channel = new Poco::ConsoleChannel(std::cerr);
Logger::root().setChannel(channel);
Logger::root().setLevel("trace");
Block block1;
{ {
Poco::AutoPtr<Poco::ConsoleChannel> channel = new Poco::ConsoleChannel(std::cerr); ColumnWithTypeAndName column1;
Logger::root().setChannel(channel); column1.name = "Sign";
Logger::root().setLevel("trace"); column1.type = std::make_shared<DataTypeInt8>();
column1.column = std::make_shared<ColumnInt8>();
column1.column->insert(DB::Int64(1));
column1.column->insert(DB::Int64(-1));
block1.insert(column1);
Block block1; ColumnWithTypeAndName column2;
column2.name = "CounterID";
{ column2.type = std::make_shared<DataTypeUInt32>();
ColumnWithTypeAndName column1; column2.column = std::make_shared<ColumnUInt32>();
column1.name = "Sign"; column2.column->insert(DB::UInt64(123));
column1.type = std::make_shared<DataTypeInt8>(); column2.column->insert(DB::UInt64(123));
column1.column = std::make_shared<ColumnInt8>(); block1.insert(column2);
column1.column->insert(DB::Int64(1));
column1.column->insert(DB::Int64(-1));
block1.insert(column1);
ColumnWithTypeAndName column2;
column2.name = "CounterID";
column2.type = std::make_shared<DataTypeUInt32>();
column2.column = std::make_shared<ColumnUInt32>();
column2.column->insert(DB::UInt64(123));
column2.column->insert(DB::UInt64(123));
block1.insert(column2);
}
Block block2;
{
ColumnWithTypeAndName column1;
column1.name = "Sign";
column1.type = std::make_shared<DataTypeInt8>();
column1.column = std::make_shared<ColumnInt8>();
column1.column->insert(DB::Int64(1));
column1.column->insert(DB::Int64(1));
block2.insert(column1);
ColumnWithTypeAndName column2;
column2.name = "CounterID";
column2.type = std::make_shared<DataTypeUInt32>();
column2.column = std::make_shared<ColumnUInt32>();
column2.column->insert(DB::UInt64(123));
column2.column->insert(DB::UInt64(456));
block2.insert(column2);
}
BlockInputStreams inputs;
inputs.push_back(std::make_shared<OneBlockInputStream>(block1));
inputs.push_back(std::make_shared<OneBlockInputStream>(block2));
SortDescription descr;
SortColumnDescription col_descr("CounterID", 1);
descr.push_back(col_descr);
//CollapsingSortedBlockInputStream collapsed(inputs, descr, "Sign", 1048576);
CollapsingFinalBlockInputStream collapsed(inputs, descr, "Sign");
Context context;
WriteBufferFromFileDescriptor out_buf(STDERR_FILENO);
BlockOutputStreamPtr output = context.getOutputFormat("TabSeparated", out_buf, block1);
copyData(collapsed, *output);
} }
catch (const Exception & e)
Block block2;
{ {
std::cerr << e.what() << ", " << e.displayText() << std::endl; ColumnWithTypeAndName column1;
return 1; column1.name = "Sign";
column1.type = std::make_shared<DataTypeInt8>();
column1.column = std::make_shared<ColumnInt8>();
column1.column->insert(DB::Int64(1));
column1.column->insert(DB::Int64(1));
block2.insert(column1);
ColumnWithTypeAndName column2;
column2.name = "CounterID";
column2.type = std::make_shared<DataTypeUInt32>();
column2.column = std::make_shared<ColumnUInt32>();
column2.column->insert(DB::UInt64(123));
column2.column->insert(DB::UInt64(456));
block2.insert(column2);
} }
BlockInputStreams inputs;
inputs.push_back(std::make_shared<OneBlockInputStream>(block1));
inputs.push_back(std::make_shared<OneBlockInputStream>(block2));
SortDescription descr;
SortColumnDescription col_descr("CounterID", 1);
descr.push_back(col_descr);
//CollapsingSortedBlockInputStream collapsed(inputs, descr, "Sign", 1048576);
CollapsingFinalBlockInputStream collapsed(inputs, descr, "Sign");
Context context;
WriteBufferFromFileDescriptor out_buf(STDERR_FILENO);
BlockOutputStreamPtr output = context.getOutputFormat("TabSeparated", out_buf, block1);
copyData(collapsed, *output);
return 0; return 0;
} }
catch (const DB::Exception & e)
{
std::cerr << e.what() << ", " << e.displayText() << std::endl;
throw;
}

View File

@ -20,65 +20,61 @@
#include <DB/Interpreters/ExpressionActions.h> #include <DB/Interpreters/ExpressionActions.h>
using Poco::SharedPtr;
int main(int argc, char ** argv) int main(int argc, char ** argv)
try
{ {
using namespace DB; using namespace DB;
try size_t n = argc == 2 ? parse<UInt64>(argv[1]) : 10ULL;
std::string input = "SELECT number, number / 3, number * number";
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "");
Context context;
ExpressionAnalyzer analyzer(ast, context, {}, {NameAndTypePair("number", std::make_shared<DataTypeUInt64>())});
ExpressionActionsChain chain;
analyzer.appendSelect(chain, false);
analyzer.appendProjectResult(chain, false);
chain.finalize();
ExpressionActionsPtr expression = chain.getLastActions();
StoragePtr table = StorageSystemNumbers::create("Numbers");
Names column_names;
column_names.push_back("number");
QueryProcessingStage::Enum stage;
BlockInputStreamPtr in;
in = table->read(column_names, 0, context, Settings(), stage)[0];
in = std::make_shared<ExpressionBlockInputStream>(in, expression);
in = std::make_shared<LimitBlockInputStream>(in, 10, std::max(static_cast<Int64>(0), static_cast<Int64>(n) - 10));
WriteBufferFromOStream out1(std::cout);
RowOutputStreamPtr out2 = std::make_shared<TabSeparatedRowOutputStream>(out1, expression->getSampleBlock());
BlockOutputStreamFromRowOutputStream out(out2);
{ {
size_t n = argc == 2 ? parse<UInt64>(argv[1]) : 10ULL; Stopwatch stopwatch;
stopwatch.start();
std::string input = "SELECT number, number / 3, number * number"; copyData(*in, out);
ParserSelectQuery parser; stopwatch.stop();
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), ""); std::cout << std::fixed << std::setprecision(2)
<< "Elapsed " << stopwatch.elapsedSeconds() << " sec."
Context context; << ", " << n / stopwatch.elapsedSeconds() << " rows/sec."
<< std::endl;
ExpressionAnalyzer analyzer(ast, context, {}, {NameAndTypePair("number", std::make_shared<DataTypeUInt64>())});
ExpressionActionsChain chain;
analyzer.appendSelect(chain, false);
analyzer.appendProjectResult(chain, false);
chain.finalize();
ExpressionActionsPtr expression = chain.getLastActions();
StoragePtr table = StorageSystemNumbers::create("Numbers");
Names column_names;
column_names.push_back("number");
QueryProcessingStage::Enum stage;
Poco::SharedPtr<IBlockInputStream> in;
in = table->read(column_names, 0, context, Settings(), stage)[0];
in = std::make_shared<ExpressionBlockInputStream>(in, expression);
in = std::make_shared<LimitBlockInputStream>(in, 10, std::max(static_cast<Int64>(0), static_cast<Int64>(n) - 10));
WriteBufferFromOStream out1(std::cout);
RowOutputStreamPtr out2 = std::make_shared<TabSeparatedRowOutputStream>(out1, expression->getSampleBlock());
BlockOutputStreamFromRowOutputStream out(out2);
{
Stopwatch stopwatch;
stopwatch.start();
copyData(*in, out);
stopwatch.stop();
std::cout << std::fixed << std::setprecision(2)
<< "Elapsed " << stopwatch.elapsedSeconds() << " sec."
<< ", " << n / stopwatch.elapsedSeconds() << " rows/sec."
<< std::endl;
}
}
catch (const Exception & e)
{
std::cerr << e.what() << ", " << e.displayText() << std::endl;
return 1;
} }
return 0; return 0;
} }
catch (const Exception & e)
{
std::cerr << e.what() << ", " << e.displayText() << std::endl;
throw;
}

View File

@ -22,69 +22,64 @@
#include <DB/Interpreters/ExpressionActions.h> #include <DB/Interpreters/ExpressionActions.h>
using Poco::SharedPtr;
int main(int argc, char ** argv) int main(int argc, char ** argv)
try
{ {
using namespace DB; using namespace DB;
try size_t n = argc == 2 ? parse<UInt64>(argv[1]) : 10ULL;
std::string input = "SELECT number, number % 3 == 1";
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "");
formatAST(*ast, std::cerr);
std::cerr << std::endl;
std::cerr << ast->getTreeID() << std::endl;
Context context;
ExpressionAnalyzer analyzer(ast, context, {}, {NameAndTypePair("number", std::make_shared<DataTypeUInt64>())});
ExpressionActionsChain chain;
analyzer.appendSelect(chain, false);
analyzer.appendProjectResult(chain, false);
chain.finalize();
ExpressionActionsPtr expression = chain.getLastActions();
StoragePtr table = StorageSystemNumbers::create("Numbers");
Names column_names;
column_names.push_back("number");
QueryProcessingStage::Enum stage;
BlockInputStreamPtr in = table->read(column_names, 0, context, Settings(), stage)[0];
in = std::make_shared<FilterBlockInputStream>(in, expression, 1);
in = std::make_shared<LimitBlockInputStream>(in, 10, std::max(static_cast<Int64>(0), static_cast<Int64>(n) - 10));
WriteBufferFromOStream ob(std::cout);
RowOutputStreamPtr out_ = std::make_shared<TabSeparatedRowOutputStream>(ob, expression->getSampleBlock());
BlockOutputStreamFromRowOutputStream out(out_);
{ {
size_t n = argc == 2 ? parse<UInt64>(argv[1]) : 10ULL; Stopwatch stopwatch;
stopwatch.start();
std::string input = "SELECT number, number % 3 == 1"; copyData(*in, out);
ParserSelectQuery parser; stopwatch.stop();
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), ""); std::cout << std::fixed << std::setprecision(2)
<< "Elapsed " << stopwatch.elapsedSeconds() << " sec."
formatAST(*ast, std::cerr); << ", " << n / stopwatch.elapsedSeconds() << " rows/sec."
std::cerr << std::endl; << std::endl;
std::cerr << ast->getTreeID() << std::endl;
Context context;
ExpressionAnalyzer analyzer(ast, context, {}, {NameAndTypePair("number", std::make_shared<DataTypeUInt64>())});
ExpressionActionsChain chain;
analyzer.appendSelect(chain, false);
analyzer.appendProjectResult(chain, false);
chain.finalize();
ExpressionActionsPtr expression = chain.getLastActions();
StoragePtr table = StorageSystemNumbers::create("Numbers");
Names column_names;
column_names.push_back("number");
QueryProcessingStage::Enum stage;
Poco::SharedPtr<IBlockInputStream> in = table->read(column_names, 0, context, Settings(), stage)[0];
in = std::make_shared<FilterBlockInputStream>(in, expression, 1);
in = std::make_shared<LimitBlockInputStream>(in, 10, std::max(static_cast<Int64>(0), static_cast<Int64>(n) - 10));
WriteBufferFromOStream ob(std::cout);
RowOutputStreamPtr out_ = std::make_shared<TabSeparatedRowOutputStream>(ob, expression->getSampleBlock());
BlockOutputStreamFromRowOutputStream out(out_);
{
Stopwatch stopwatch;
stopwatch.start();
copyData(*in, out);
stopwatch.stop();
std::cout << std::fixed << std::setprecision(2)
<< "Elapsed " << stopwatch.elapsedSeconds() << " sec."
<< ", " << n / stopwatch.elapsedSeconds() << " rows/sec."
<< std::endl;
}
}
catch (const Exception & e)
{
std::cerr << e.what() << ", " << e.displayText() << std::endl;
return 1;
} }
return 0; return 0;
} }
catch (const Exception & e)
{
std::cerr << e.what() << ", " << e.displayText() << std::endl;
throw;
}

View File

@ -125,7 +125,7 @@ int main(int argc, char ** argv)
QueryProcessingStage::Enum stage; QueryProcessingStage::Enum stage;
Poco::SharedPtr<IBlockInputStream> in = table->read(column_names, 0, context, Settings(), stage)[0]; BlockInputStreamPtr in = table->read(column_names, 0, context, Settings(), stage)[0];
in = std::make_shared<FilterBlockInputStream>(in, expression, 4); in = std::make_shared<FilterBlockInputStream>(in, expression, 4);
//in = std::make_shared<LimitBlockInputStream>(in, 10, 0); //in = std::make_shared<LimitBlockInputStream>(in, 10, 0);

View File

@ -23,9 +23,6 @@
#include <DB/Interpreters/ExpressionActions.h> #include <DB/Interpreters/ExpressionActions.h>
using Poco::SharedPtr;
void thread1(DB::BlockInputStreamPtr in, DB::BlockOutputStreamPtr out, DB::WriteBuffer & out_buf) void thread1(DB::BlockInputStreamPtr in, DB::BlockOutputStreamPtr out, DB::WriteBuffer & out_buf)
{ {
while (DB::Block block = in->read()) while (DB::Block block = in->read())
@ -46,69 +43,67 @@ void thread2(DB::BlockInputStreamPtr in, DB::BlockOutputStreamPtr out, DB::Write
int main(int argc, char ** argv) int main(int argc, char ** argv)
try
{ {
using namespace DB; using namespace DB;
try std::string input = "SELECT number, number % 10000000 == 1";
{
std::string input = "SELECT number, number % 10000000 == 1";
ParserSelectQuery parser; ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), ""); ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "");
formatAST(*ast, std::cerr); formatAST(*ast, std::cerr);
std::cerr << std::endl; std::cerr << std::endl;
Context context; Context context;
ExpressionAnalyzer analyzer(ast, context, {}, {NameAndTypePair("number", std::make_shared<DataTypeUInt64>())}); ExpressionAnalyzer analyzer(ast, context, {}, {NameAndTypePair("number", std::make_shared<DataTypeUInt64>())});
ExpressionActionsChain chain; ExpressionActionsChain chain;
analyzer.appendSelect(chain, false); analyzer.appendSelect(chain, false);
analyzer.appendProjectResult(chain, false); analyzer.appendProjectResult(chain, false);
chain.finalize(); chain.finalize();
ExpressionActionsPtr expression = chain.getLastActions(); ExpressionActionsPtr expression = chain.getLastActions();
StoragePtr table = StorageSystemNumbers::create("Numbers"); StoragePtr table = StorageSystemNumbers::create("Numbers");
Names column_names; Names column_names;
column_names.push_back("number"); column_names.push_back("number");
QueryProcessingStage::Enum stage; QueryProcessingStage::Enum stage;
BlockInputStreamPtr in = table->read(column_names, 0, context, Settings(), stage)[0]; BlockInputStreamPtr in = table->read(column_names, 0, context, Settings(), stage)[0];
ForkBlockInputStreams fork(in); ForkBlockInputStreams fork(in);
BlockInputStreamPtr in1 = fork.createInput(); BlockInputStreamPtr in1 = fork.createInput();
BlockInputStreamPtr in2 = fork.createInput(); BlockInputStreamPtr in2 = fork.createInput();
in1 = std::make_shared<FilterBlockInputStream>(in1, expression, 1); in1 = std::make_shared<FilterBlockInputStream>(in1, expression, 1);
in1 = std::make_shared<LimitBlockInputStream>(in1, 10, 0); in1 = std::make_shared<LimitBlockInputStream>(in1, 10, 0);
in2 = std::make_shared<FilterBlockInputStream>(in2, expression, 1); in2 = std::make_shared<FilterBlockInputStream>(in2, expression, 1);
in2 = std::make_shared<LimitBlockInputStream>(in2, 20, 5); in2 = std::make_shared<LimitBlockInputStream>(in2, 20, 5);
Block out_sample = expression->getSampleBlock(); Block out_sample = expression->getSampleBlock();
WriteBufferFromOStream ob1(std::cout); WriteBufferFromOStream ob1(std::cout);
WriteBufferFromOStream ob2(std::cerr); WriteBufferFromOStream ob2(std::cerr);
BlockOutputStreamPtr out1 = context.getOutputFormat("TabSeparated", ob1, out_sample); BlockOutputStreamPtr out1 = context.getOutputFormat("TabSeparated", ob1, out_sample);
BlockOutputStreamPtr out2 = context.getOutputFormat("TabSeparated", ob2, out_sample); BlockOutputStreamPtr out2 = context.getOutputFormat("TabSeparated", ob2, out_sample);
std::thread thr1(std::bind(thread1, in1, out1, std::ref(ob1))); std::thread thr1(std::bind(thread1, in1, out1, std::ref(ob1)));
std::thread thr2(std::bind(thread2, in2, out2, std::ref(ob2))); std::thread thr2(std::bind(thread2, in2, out2, std::ref(ob2)));
fork.run(); fork.run();
thr1.join(); thr1.join();
thr2.join(); thr2.join();
}
catch (const Exception & e)
{
std::cerr << e.what() << ", " << e.displayText() << std::endl;
return 1;
}
return 0; return 0;
} }
catch (const Exception & e)
{
std::cerr << e.what() << ", " << e.displayText() << std::endl;
throw;
}

View File

@ -11,7 +11,6 @@
#include <DB/DataStreams/glueBlockInputStreams.h> #include <DB/DataStreams/glueBlockInputStreams.h>
using Poco::SharedPtr;
using namespace DB; using namespace DB;
@ -33,69 +32,66 @@ void forkThread(ForkPtr fork)
int main(int argc, char ** argv) int main(int argc, char ** argv)
try
{ {
try Context context;
{
Context context;
context.setGlobalContext(context); context.setGlobalContext(context);
context.setPath("./"); context.setPath("./");
loadMetadata(context); loadMetadata(context);
context.setCurrentDatabase("default"); context.setCurrentDatabase("default");
context.setSetting("max_threads", 1UL); context.setSetting("max_threads", 1UL);
BlockIO io1 = executeQuery( BlockIO io1 = executeQuery(
"SELECT SearchPhrase, count()" "SELECT SearchPhrase, count()"
" FROM hits" " FROM hits"
" WHERE SearchPhrase != ''" " WHERE SearchPhrase != ''"
" GROUP BY SearchPhrase" " GROUP BY SearchPhrase"
" ORDER BY count() DESC" " ORDER BY count() DESC"
" LIMIT 10", " LIMIT 10",
context, QueryProcessingStage::Complete); context, QueryProcessingStage::Complete);
BlockIO io2 = executeQuery( BlockIO io2 = executeQuery(
"SELECT count()" "SELECT count()"
" FROM hits" " FROM hits"
" WHERE SearchPhrase != ''", " WHERE SearchPhrase != ''",
context, QueryProcessingStage::Complete); context, QueryProcessingStage::Complete);
WriteBufferFromFileDescriptor wb(STDOUT_FILENO); WriteBufferFromFileDescriptor wb(STDOUT_FILENO);
BlockOutputStreamPtr out1 = context.getOutputFormat("TabSeparated", wb, io1.in_sample); BlockOutputStreamPtr out1 = context.getOutputFormat("TabSeparated", wb, io1.in_sample);
BlockOutputStreamPtr out2 = context.getOutputFormat("TabSeparated", wb, io2.in_sample); BlockOutputStreamPtr out2 = context.getOutputFormat("TabSeparated", wb, io2.in_sample);
BlockInputStreams inputs; BlockInputStreams inputs;
inputs.push_back(io1.in); inputs.push_back(io1.in);
inputs.push_back(io2.in); inputs.push_back(io2.in);
for (size_t i = 0; i < inputs.size(); ++i) for (size_t i = 0; i < inputs.size(); ++i)
std::cerr << inputs[i]->getID() << std::endl; std::cerr << inputs[i]->getID() << std::endl;
Forks forks; Forks forks;
glueBlockInputStreams(inputs, forks); glueBlockInputStreams(inputs, forks);
std::cerr << forks.size() << std::endl; std::cerr << forks.size() << std::endl;
std::mutex mutex; std::mutex mutex;
boost::threadpool::pool pool(inputs.size() + forks.size()); boost::threadpool::pool pool(inputs.size() + forks.size());
pool.schedule(std::bind(inputThread, inputs[0], out1, std::ref(wb), std::ref(mutex))); pool.schedule(std::bind(inputThread, inputs[0], out1, std::ref(wb), std::ref(mutex)));
pool.schedule(std::bind(inputThread, inputs[1], out2, std::ref(wb), std::ref(mutex))); pool.schedule(std::bind(inputThread, inputs[1], out2, std::ref(wb), std::ref(mutex)));
for (size_t i = 0; i < forks.size(); ++i) for (size_t i = 0; i < forks.size(); ++i)
pool.schedule(std::bind(forkThread, forks[i])); pool.schedule(std::bind(forkThread, forks[i]));
pool.wait();
}
catch (const DB::Exception & e)
{
std::cerr << e.what() << ", " << e.displayText() << std::endl;
return 1;
}
pool.wait();
return 0; return 0;
} }
catch (const DB::Exception & e)
{
std::cerr << e.what() << ", " << e.displayText() << std::endl;
throw;
}

View File

@ -101,7 +101,7 @@ try
if (argc == 2 && 0 == strcmp(argv[1], "read")) if (argc == 2 && 0 == strcmp(argv[1], "read"))
{ {
QueryProcessingStage::Enum stage; QueryProcessingStage::Enum stage;
SharedPtr<IBlockInputStream> in = table->read(column_names, 0, Context{}, Settings(), stage)[0]; BlockInputStreamPtr in = table->read(column_names, 0, Context{}, Settings(), stage)[0];
WriteBufferFromFileDescriptor out1(STDOUT_FILENO); WriteBufferFromFileDescriptor out1(STDOUT_FILENO);
CompressedWriteBuffer out2(out1); CompressedWriteBuffer out2(out1);
NativeBlockOutputStream out3(out2, ClickHouseRevision::get()); NativeBlockOutputStream out3(out2, ClickHouseRevision::get());
@ -114,7 +114,7 @@ try
ReadBufferFromFileDescriptor in1(STDIN_FILENO); ReadBufferFromFileDescriptor in1(STDIN_FILENO);
CompressedReadBuffer in2(in1); CompressedReadBuffer in2(in1);
NativeBlockInputStream in3(in2, ClickHouseRevision::get()); NativeBlockInputStream in3(in2, ClickHouseRevision::get());
SharedPtr<IBlockOutputStream> out = table->write({}, {}); BlockOutputStreamPtr out = table->write({}, {});
copyData(in3, *out); copyData(in3, *out);
} }

View File

@ -143,7 +143,7 @@ try
QueryProcessingStage::Enum stage; QueryProcessingStage::Enum stage;
Poco::SharedPtr<IBlockInputStream> in = table->read(column_names, 0, Context{}, Settings(), stage, argc == 2 ? atoi(argv[1]) : 1048576)[0]; BlockInputStreamPtr in = table->read(column_names, 0, Context{}, Settings(), stage, argc == 2 ? atoi(argv[1]) : 1048576)[0];
in = std::make_shared<PartialSortingBlockInputStream>(in, sort_columns); in = std::make_shared<PartialSortingBlockInputStream>(in, sort_columns);
in = std::make_shared<MergeSortingBlockInputStream>(in, sort_columns, DEFAULT_BLOCK_SIZE, 0, 0, ""); in = std::make_shared<MergeSortingBlockInputStream>(in, sort_columns, DEFAULT_BLOCK_SIZE, 0, 0, "");
//in = std::make_shared<LimitBlockInputStream>(in, 10, 0); //in = std::make_shared<LimitBlockInputStream>(in, 10, 0);

View File

@ -12,9 +12,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
std::string DataTypeAggregateFunction::getName() const std::string DataTypeAggregateFunction::getName() const
{ {
std::stringstream stream; std::stringstream stream;

View File

@ -1,5 +1,3 @@
#include <Poco/SharedPtr.h>
#include <DB/IO/WriteBuffer.h> #include <DB/IO/WriteBuffer.h>
#include <DB/IO/WriteHelpers.h> #include <DB/IO/WriteHelpers.h>
@ -17,8 +15,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int CANNOT_READ_ALL_DATA; extern const int CANNOT_READ_ALL_DATA;

View File

@ -1,5 +1,3 @@
#include <Poco/SharedPtr.h>
#include <DB/Core/Defines.h> #include <DB/Core/Defines.h>
#include <DB/Columns/ColumnString.h> #include <DB/Columns/ColumnString.h>

View File

@ -12,60 +12,60 @@
int main(int argc, char ** argv) int main(int argc, char ** argv)
try
{ {
try using namespace DB;
Stopwatch stopwatch;
size_t n = 50000000;
const char * s = "";
size_t size = strlen(s) + 1;
DataTypeString data_type;
{ {
Stopwatch stopwatch; std::shared_ptr<ColumnString> column = std::make_shared<ColumnString>();
size_t n = 50000000; ColumnString::Chars_t & data = column->getChars();
const char * s = ""; ColumnString::Offsets_t & offsets = column->getOffsets();
size_t size = strlen(s) + 1;
DB::DataTypeString data_type;
data.resize(n * size);
offsets.resize(n);
for (size_t i = 0; i < n; ++i)
{ {
Poco::SharedPtr<DB::ColumnString> column = new DB::ColumnString(); memcpy(&data[i * size], s, size);
DB::ColumnString::Chars_t & data = column->getChars(); offsets[i] = (i + 1) * size;
DB::ColumnString::Offsets_t & offsets = column->getOffsets();
data.resize(n * size);
offsets.resize(n);
for (size_t i = 0; i < n; ++i)
{
memcpy(&data[i * size], s, size);
offsets[i] = (i + 1) * size;
}
std::ofstream ostr("test");
DB::WriteBufferFromOStream out_buf(ostr);
stopwatch.restart();
data_type.serializeBinary(*column, out_buf);
stopwatch.stop();
std::cout << "Writing, elapsed: " << stopwatch.elapsedSeconds() << std::endl;
} }
{ std::ofstream ostr("test");
Poco::SharedPtr<DB::ColumnString> column = new DB::ColumnString(); WriteBufferFromOStream out_buf(ostr);
std::ifstream istr("test"); stopwatch.restart();
DB::ReadBufferFromIStream in_buf(istr); data_type.serializeBinary(*column, out_buf);
stopwatch.stop();
stopwatch.restart(); std::cout << "Writing, elapsed: " << stopwatch.elapsedSeconds() << std::endl;
data_type.deserializeBinary(*column, in_buf, n, 0);
stopwatch.stop();
std::cout << "Reading, elapsed: " << stopwatch.elapsedSeconds() << std::endl;
std::cout << std::endl
<< DB::get<const DB::String &>((*column)[0]) << std::endl
<< DB::get<const DB::String &>((*column)[n - 1]) << std::endl;
}
} }
catch (const DB::Exception & e)
{ {
std::cerr << e.what() << ", " << e.displayText() << std::endl; std::shared_ptr<ColumnString> column = std::make_shared<ColumnString>();
return 1;
std::ifstream istr("test");
ReadBufferFromIStream in_buf(istr);
stopwatch.restart();
data_type.deserializeBinary(*column, in_buf, n, 0);
stopwatch.stop();
std::cout << "Reading, elapsed: " << stopwatch.elapsedSeconds() << std::endl;
std::cout << std::endl
<< get<const String &>((*column)[0]) << std::endl
<< get<const String &>((*column)[n - 1]) << std::endl;
} }
return 0; return 0;
} }
catch (const DB::Exception & e)
{
std::cerr << e.what() << ", " << e.displayText() << std::endl;
return 1;
}

View File

@ -10,9 +10,11 @@
int main(int argc, char ** argv) int main(int argc, char ** argv)
{ {
Poco::SharedPtr<DB::ColumnUInt64> column = new DB::ColumnUInt64(); using namespace DB;
DB::ColumnUInt64::Container_t & vec = column->getData();
DB::DataTypeUInt64 data_type; std::shared_ptr<ColumnUInt64> column = std::make_shared<ColumnUInt64>();
ColumnUInt64::Container_t & vec = column->getData();
DataTypeUInt64 data_type;
Stopwatch stopwatch; Stopwatch stopwatch;
size_t n = 10000000; size_t n = 10000000;
@ -22,7 +24,7 @@ int main(int argc, char ** argv)
vec[i] = i; vec[i] = i;
std::ofstream ostr("test"); std::ofstream ostr("test");
DB::WriteBufferFromOStream out_buf(ostr); WriteBufferFromOStream out_buf(ostr);
stopwatch.restart(); stopwatch.restart();
data_type.serializeBinary(*column, out_buf); data_type.serializeBinary(*column, out_buf);

View File

@ -2,7 +2,6 @@
#include <set> #include <set>
#include <chrono> #include <chrono>
#include <Poco/SharedPtr.h>
#include <Poco/Mutex.h> #include <Poco/Mutex.h>
#include <Poco/File.h> #include <Poco/File.h>
#include <Poco/UUIDGenerator.h> #include <Poco/UUIDGenerator.h>
@ -66,8 +65,6 @@ namespace ErrorCodes
class TableFunctionFactory; class TableFunctionFactory;
using Poco::SharedPtr;
/** Набор известных объектов, которые могут быть использованы в запросе. /** Набор известных объектов, которые могут быть использованы в запросе.
* Разделяемая часть. Порядок членов (порядок их уничтожения) очень важен. * Разделяемая часть. Порядок членов (порядок их уничтожения) очень важен.
@ -94,8 +91,8 @@ struct ContextShared
TableFunctionFactory table_function_factory; /// Табличные функции. TableFunctionFactory table_function_factory; /// Табличные функции.
AggregateFunctionFactory aggregate_function_factory; /// Агрегатные функции. AggregateFunctionFactory aggregate_function_factory; /// Агрегатные функции.
FormatFactory format_factory; /// Форматы. FormatFactory format_factory; /// Форматы.
mutable SharedPtr<Dictionaries> dictionaries; /// Словари Метрики. Инициализируются лениво. mutable std::shared_ptr<Dictionaries> dictionaries; /// Словари Метрики. Инициализируются лениво.
mutable SharedPtr<ExternalDictionaries> external_dictionaries; mutable std::shared_ptr<ExternalDictionaries> external_dictionaries;
Users users; /// Известные пользователи. Users users; /// Известные пользователи.
Quotas quotas; /// Известные квоты на использование ресурсов. Quotas quotas; /// Известные квоты на использование ресурсов.
mutable UncompressedCachePtr uncompressed_cache; /// Кэш разжатых блоков. mutable UncompressedCachePtr uncompressed_cache; /// Кэш разжатых блоков.
@ -116,7 +113,7 @@ struct ContextShared
/// Кластеры для distributed таблиц /// Кластеры для distributed таблиц
/// Создаются при создании Distributed таблиц, так как нужно дождаться пока будут выставлены Settings /// Создаются при создании Distributed таблиц, так как нужно дождаться пока будут выставлены Settings
mutable Poco::SharedPtr<Clusters> clusters; mutable std::shared_ptr<Clusters> clusters;
Poco::UUIDGenerator uuid_generator; Poco::UUIDGenerator uuid_generator;
@ -747,7 +744,7 @@ const Dictionaries & Context::getDictionariesImpl(const bool throw_on_error) con
auto lock = getLock(); auto lock = getLock();
if (!shared->dictionaries) if (!shared->dictionaries)
shared->dictionaries = new Dictionaries{throw_on_error}; shared->dictionaries = std::make_shared<Dictionaries>(throw_on_error);
return *shared->dictionaries; return *shared->dictionaries;
} }
@ -761,7 +758,7 @@ const ExternalDictionaries & Context::getExternalDictionariesImpl(const bool thr
{ {
if (!this->global_context) if (!this->global_context)
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR); throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
shared->external_dictionaries = new ExternalDictionaries{*this->global_context, throw_on_error}; shared->external_dictionaries = std::make_shared<ExternalDictionaries>(*this->global_context, throw_on_error);
} }
return *shared->external_dictionaries; return *shared->external_dictionaries;
@ -921,7 +918,7 @@ const Cluster & Context::getCluster(const std::string & cluster_name) const
{ {
auto lock = getLock(); auto lock = getLock();
if (!shared->clusters) if (!shared->clusters)
shared->clusters = new Clusters(settings); shared->clusters = std::make_shared<Clusters>(settings);
} }
Clusters::Impl::iterator it = shared->clusters->impl.find(cluster_name); Clusters::Impl::iterator it = shared->clusters->impl.find(cluster_name);
@ -931,12 +928,12 @@ const Cluster & Context::getCluster(const std::string & cluster_name) const
throw Poco::Exception("Failed to find cluster with name = " + cluster_name); throw Poco::Exception("Failed to find cluster with name = " + cluster_name);
} }
Poco::SharedPtr<Clusters> Context::getClusters() const std::shared_ptr<Clusters> Context::getClusters() const
{ {
{ {
auto lock = getLock(); auto lock = getLock();
if (!shared->clusters) if (!shared->clusters)
shared->clusters = new Clusters(settings); shared->clusters = std::make_shared<Clusters>(settings);
} }
return shared->clusters; return shared->clusters;
@ -947,7 +944,7 @@ Compiler & Context::getCompiler()
auto lock = getLock(); auto lock = getLock();
if (!shared->compiler) if (!shared->compiler)
shared->compiler.reset(new Compiler{ shared->path + "build/", 1 }); shared->compiler = std::make_unique<Compiler>(shared->path + "build/", 1);
return *shared->compiler; return *shared->compiler;
} }

View File

@ -15,7 +15,7 @@ using ColumnsWithSortDescriptions = std::vector<std::pair<const IColumn *, SortC
static inline bool needCollation(const IColumn * column, const SortColumnDescription & description) static inline bool needCollation(const IColumn * column, const SortColumnDescription & description)
{ {
if (description.collator.isNull()) if (!description.collator)
return false; return false;
if (column->getName() != "ColumnString") if (column->getName() != "ColumnString")

View File

@ -777,7 +777,7 @@ bool ParserOrderByElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & ma
ws.ignore(pos, end); ws.ignore(pos, end);
Poco::SharedPtr<Collator> collator = nullptr; std::shared_ptr<Collator> collator;
if (collate.ignore(pos, end)) if (collate.ignore(pos, end))
{ {
ws.ignore(pos, end); ws.ignore(pos, end);
@ -787,7 +787,7 @@ bool ParserOrderByElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & ma
return false; return false;
const String & locale = typeid_cast<const ASTLiteral &>(*locale_node).value.safeGet<String>(); const String & locale = typeid_cast<const ASTLiteral &>(*locale_node).value.safeGet<String>();
collator = new Collator(locale); collator = std::make_shared<Collator>(locale);
} }
node = new ASTOrderByElement(StringRange(begin, pos), direction, collator); node = new ASTOrderByElement(StringRange(begin, pos), direction, collator);

View File

@ -74,13 +74,14 @@ void HTTPHandler::processQuery(Poco::Net::HTTPServerRequest & request, Poco::Net
} }
} }
used_output.out = new WriteBufferFromHTTPServerResponse(response, client_supports_http_compression, http_response_compression_method); used_output.out = std::make_shared<WriteBufferFromHTTPServerResponse>(
response, client_supports_http_compression, http_response_compression_method);
/** Клиент может указать compress в query string. /** Клиент может указать compress в query string.
* В этом случае, результат сжимается несовместимым алгоритмом для внутреннего использования и этот факт не отражается в HTTP заголовках. * В этом случае, результат сжимается несовместимым алгоритмом для внутреннего использования и этот факт не отражается в HTTP заголовках.
*/ */
if (parse<bool>(params.get("compress", "0"))) if (parse<bool>(params.get("compress", "0")))
used_output.out_maybe_compressed = new CompressedWriteBuffer(*used_output.out); used_output.out_maybe_compressed = std::make_shared<CompressedWriteBuffer>(*used_output.out);
else else
used_output.out_maybe_compressed = used_output.out; used_output.out_maybe_compressed = used_output.out;

View File

@ -20,9 +20,9 @@ public:
struct Output struct Output
{ {
SharedPtr<WriteBufferFromHTTPServerResponse> out; std::shared_ptr<WriteBufferFromHTTPServerResponse> out;
/// Используется для выдачи ответа. Равен либо out, либо CompressedWriteBuffer(*out), в зависимости от настроек. /// Используется для выдачи ответа. Равен либо out, либо CompressedWriteBuffer(*out), в зависимости от настроек.
SharedPtr<WriteBuffer> out_maybe_compressed; std::shared_ptr<WriteBuffer> out_maybe_compressed;
}; };
void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response); void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response);

View File

@ -617,7 +617,7 @@ using SocialSourceNetworkID = AttributeUIntBase;
/** Информация о типах атрибутов */ /** Информация о типах атрибутов */
using AttributeMetadatas = std::map<std::string, Poco::SharedPtr<IAttributeMetadata> >; using AttributeMetadatas = std::map<std::string, Poco::SharedPtr<IAttributeMetadata>>;
inline AttributeMetadatas GetOLAPAttributeMetadata() inline AttributeMetadatas GetOLAPAttributeMetadata()
{ {

View File

@ -11,6 +11,7 @@
#include <ext/scope_guard.hpp> #include <ext/scope_guard.hpp>
#include <memory> #include <memory>
#include <experimental/optional>
#include <DB/Common/Macros.h> #include <DB/Common/Macros.h>
#include <DB/Common/getFQDNOrHostName.h> #include <DB/Common/getFQDNOrHostName.h>
@ -390,13 +391,13 @@ int Server::main(const std::vector<std::string> & args)
new Poco::Net::TCPServerParams); new Poco::Net::TCPServerParams);
/// Interserver IO HTTP /// Interserver IO HTTP
Poco::SharedPtr<Poco::Net::HTTPServer> interserver_io_http_server; std::experimental::optional<Poco::Net::HTTPServer> interserver_io_http_server;
if (config().has("interserver_http_port")) if (config().has("interserver_http_port"))
{ {
Poco::Net::ServerSocket interserver_io_http_socket(Poco::Net::SocketAddress(listen_host, config().getInt("interserver_http_port"))); Poco::Net::ServerSocket interserver_io_http_socket(Poco::Net::SocketAddress(listen_host, config().getInt("interserver_http_port")));
interserver_io_http_socket.setReceiveTimeout(settings.receive_timeout); interserver_io_http_socket.setReceiveTimeout(settings.receive_timeout);
interserver_io_http_socket.setSendTimeout(settings.send_timeout); interserver_io_http_socket.setSendTimeout(settings.send_timeout);
interserver_io_http_server = new Poco::Net::HTTPServer( interserver_io_http_server.reset(
new HTTPRequestHandlerFactory<InterserverIOHTTPHandler>(*this, "InterserverIOHTTPHandler-factory"), new HTTPRequestHandlerFactory<InterserverIOHTTPHandler>(*this, "InterserverIOHTTPHandler-factory"),
server_pool, server_pool,
interserver_io_http_socket, interserver_io_http_socket,
@ -404,7 +405,7 @@ int Server::main(const std::vector<std::string> & args)
} }
/// OLAP HTTP /// OLAP HTTP
Poco::SharedPtr<Poco::Net::HTTPServer> olap_http_server; std::experimental::optional<Poco::Net::HTTPServer> olap_http_server;
bool use_olap_server = config().has("olap_compatibility.port"); bool use_olap_server = config().has("olap_compatibility.port");
if (use_olap_server) if (use_olap_server)
{ {
@ -414,7 +415,7 @@ int Server::main(const std::vector<std::string> & args)
Poco::Net::ServerSocket olap_http_socket(Poco::Net::SocketAddress(listen_host, config().getInt("olap_compatibility.port"))); Poco::Net::ServerSocket olap_http_socket(Poco::Net::SocketAddress(listen_host, config().getInt("olap_compatibility.port")));
olap_http_socket.setReceiveTimeout(settings.receive_timeout); olap_http_socket.setReceiveTimeout(settings.receive_timeout);
olap_http_socket.setSendTimeout(settings.send_timeout); olap_http_socket.setSendTimeout(settings.send_timeout);
olap_http_server = new Poco::Net::HTTPServer( olap_http_server.reset(
new HTTPRequestHandlerFactory<OLAPHTTPHandler>(*this, "OLAPHTTPHandler-factory"), new HTTPRequestHandlerFactory<OLAPHTTPHandler>(*this, "OLAPHTTPHandler-factory"),
server_pool, server_pool,
olap_http_socket, olap_http_socket,

View File

@ -56,8 +56,8 @@ void TCPHandler::runImpl()
socket().setSendTimeout(global_settings.send_timeout); socket().setSendTimeout(global_settings.send_timeout);
socket().setNoDelay(true); socket().setNoDelay(true);
in = new ReadBufferFromPocoSocket(socket()); in = std::make_shared<ReadBufferFromPocoSocket>(socket());
out = new WriteBufferFromPocoSocket(socket()); out = std::make_shared<WriteBufferFromPocoSocket>(socket());
if (in->eof()) if (in->eof())
{ {
@ -128,7 +128,7 @@ void TCPHandler::runImpl()
/** Исключение во время выполнения запроса (его надо отдать по сети клиенту). /** Исключение во время выполнения запроса (его надо отдать по сети клиенту).
* Клиент сможет его принять, если оно не произошло во время отправки другого пакета и клиент ещё не разорвал соединение. * Клиент сможет его принять, если оно не произошло во время отправки другого пакета и клиент ещё не разорвал соединение.
*/ */
SharedPtr<Exception> exception; std::unique_ptr<Exception> exception;
try try
{ {
@ -148,8 +148,8 @@ void TCPHandler::runImpl()
/// Очищаем, так как, получая данные внешних таблиц, мы получили пустой блок. /// Очищаем, так как, получая данные внешних таблиц, мы получили пустой блок.
/// А значит, stream помечен как cancelled и читать из него нельзя. /// А значит, stream помечен как cancelled и читать из него нельзя.
state.block_in = nullptr; state.block_in.reset();
state.maybe_compressed_in = nullptr; /// Для более корректного учёта MemoryTracker-ом. state.maybe_compressed_in.reset(); /// Для более корректного учёта MemoryTracker-ом.
/// Обрабатываем Query /// Обрабатываем Query
state.io = executeQuery(state.query, query_context, false, state.stage); state.io = executeQuery(state.query, query_context, false, state.stage);
@ -173,7 +173,7 @@ void TCPHandler::runImpl()
catch (const Exception & e) catch (const Exception & e)
{ {
state.io.onException(); state.io.onException();
exception = e.clone(); exception.reset(e.clone());
if (e.code() == ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT) if (e.code() == ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT)
throw; throw;
@ -187,22 +187,22 @@ void TCPHandler::runImpl()
* Будем пытаться отправить эксепшен клиенту в любом случае - см. ниже. * Будем пытаться отправить эксепшен клиенту в любом случае - см. ниже.
*/ */
state.io.onException(); state.io.onException();
exception = new Exception(e.displayText(), ErrorCodes::POCO_EXCEPTION); exception = std::make_unique<Exception>(e.displayText(), ErrorCodes::POCO_EXCEPTION);
} }
catch (const Poco::Exception & e) catch (const Poco::Exception & e)
{ {
state.io.onException(); state.io.onException();
exception = new Exception(e.displayText(), ErrorCodes::POCO_EXCEPTION); exception = std::make_unique<Exception>(e.displayText(), ErrorCodes::POCO_EXCEPTION);
} }
catch (const std::exception & e) catch (const std::exception & e)
{ {
state.io.onException(); state.io.onException();
exception = new Exception(e.what(), ErrorCodes::STD_EXCEPTION); exception = std::make_unique<Exception>(e.what(), ErrorCodes::STD_EXCEPTION);
} }
catch (...) catch (...)
{ {
state.io.onException(); state.io.onException();
exception = new Exception("Unknown exception", ErrorCodes::UNKNOWN_EXCEPTION); exception = std::make_unique<Exception>("Unknown exception", ErrorCodes::UNKNOWN_EXCEPTION);
} }
bool network_error = false; bool network_error = false;
@ -609,7 +609,7 @@ void TCPHandler::initBlockInput()
if (!state.block_in) if (!state.block_in)
{ {
if (state.compression == Protocol::Compression::Enable) if (state.compression == Protocol::Compression::Enable)
state.maybe_compressed_in = new CompressedReadBuffer(*in); state.maybe_compressed_in = std::make_shared<CompressedReadBuffer>(*in);
else else
state.maybe_compressed_in = in; state.maybe_compressed_in = in;
@ -625,7 +625,8 @@ void TCPHandler::initBlockOutput()
if (!state.block_out) if (!state.block_out)
{ {
if (state.compression == Protocol::Compression::Enable) if (state.compression == Protocol::Compression::Enable)
state.maybe_compressed_out = new CompressedWriteBuffer(*out, query_context.getSettings().network_compression_method); state.maybe_compressed_out = std::make_shared<CompressedWriteBuffer>(
*out, query_context.getSettings().network_compression_method);
else else
state.maybe_compressed_out = out; state.maybe_compressed_out = out;

View File

@ -28,11 +28,11 @@ struct QueryState
Protocol::Compression::Enum compression = Protocol::Compression::Disable; Protocol::Compression::Enum compression = Protocol::Compression::Disable;
/// Откуда читать данные для INSERT-а. /// Откуда читать данные для INSERT-а.
SharedPtr<ReadBuffer> maybe_compressed_in; shared_ptr<ReadBuffer> maybe_compressed_in;
BlockInputStreamPtr block_in; BlockInputStreamPtr block_in;
/// Куда писать возвращаемые данные. /// Куда писать возвращаемые данные.
SharedPtr<WriteBuffer> maybe_compressed_out; shared_ptr<WriteBuffer> maybe_compressed_out;
BlockOutputStreamPtr block_out; BlockOutputStreamPtr block_out;
/// Текст запроса. /// Текст запроса.
@ -87,8 +87,8 @@ private:
Context query_context; Context query_context;
/// Потоки для чтения/записи из/в сокет соединения с клиентом. /// Потоки для чтения/записи из/в сокет соединения с клиентом.
SharedPtr<ReadBuffer> in; std::shared_ptr<ReadBuffer> in;
SharedPtr<WriteBuffer> out; std::shared_ptr<WriteBuffer> out;
/// Время после последней проверки остановки запроса и отправки прогресса. /// Время после последней проверки остановки запроса и отправки прогресса.
Stopwatch after_check_cancelled; Stopwatch after_check_cancelled;

View File

@ -150,7 +150,7 @@ StoragePtr StorageDistributed::create(
NamesAndTypesListPtr columns_, NamesAndTypesListPtr columns_,
const String & remote_database_, const String & remote_database_,
const String & remote_table_, const String & remote_table_,
SharedPtr<Cluster> & owned_cluster_, std::shared_ptr<Cluster> & owned_cluster_,
Context & context_) Context & context_)
{ {
auto res = new StorageDistributed{ auto res = new StorageDistributed{

View File

@ -32,8 +32,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;

View File

@ -10,9 +10,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
class MemoryBlockInputStream : public IProfilingBlockInputStream class MemoryBlockInputStream : public IProfilingBlockInputStream
{ {
public: public:

View File

@ -31,8 +31,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int EMPTY_LIST_OF_COLUMNS_PASSED; extern const int EMPTY_LIST_OF_COLUMNS_PASSED;

View File

@ -1,5 +1,3 @@
#include <Poco/SharedPtr.h>
#include <DB/Common/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Columns/ColumnsNumber.h> #include <DB/Columns/ColumnsNumber.h>
#include <DB/DataTypes/DataTypesNumberFixed.h> #include <DB/DataTypes/DataTypesNumberFixed.h>
@ -10,8 +8,6 @@
namespace DB namespace DB
{ {
using Poco::SharedPtr;
class NumbersBlockInputStream : public IProfilingBlockInputStream class NumbersBlockInputStream : public IProfilingBlockInputStream
{ {
public: public:

View File

@ -14,119 +14,115 @@
#include <DB/Parsers/ParserSelectQuery.h> #include <DB/Parsers/ParserSelectQuery.h>
#include <DB/Parsers/parseQuery.h> #include <DB/Parsers/parseQuery.h>
using Poco::SharedPtr;
int main(int argc, char ** argv) int main(int argc, char ** argv)
try
{ {
using namespace DB; using namespace DB;
try const size_t rows = 12345;
Context context;
/// создаём таблицу с парой столбцов
NamesAndTypesListPtr names_and_types = std::make_shared<NamesAndTypesList>();
names_and_types->push_back(NameAndTypePair("d", std::make_shared<DataTypeDate>()));
names_and_types->push_back(NameAndTypePair("a", std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt32>())));
ASTPtr primary_expr;
Expected expected = "";
String primary_expr_str = "d";
const char * begin = primary_expr_str.data();
const char * end = begin + primary_expr_str.size();
const char * max_parsed_pos = begin;
ParserExpressionList parser(false);
if (!parser.parse(begin, end, primary_expr, max_parsed_pos, expected))
throw Poco::Exception("Cannot parse " + primary_expr_str);
MergeTreeData::MergingParams params;
params.mode = MergeTreeData::MergingParams::Ordinary;
StoragePtr table = StorageMergeTree::create(
"./", "default", "test",
names_and_types, {}, {}, ColumnDefaults{},
context, primary_expr, "d",
nullptr, 101, params, {});
/// пишем в неё
{ {
const size_t rows = 12345; Block block;
Context context; ColumnWithTypeAndName column1;
column1.name = "d";
column1.type = table->getDataTypeByName("d");
column1.column = column1.type->createColumn();
ColumnUInt16::Container_t & vec1 = typeid_cast<ColumnUInt16 &>(*column1.column).getData();
/// создаём таблицу с парой столбцов vec1.resize(rows);
for (size_t i = 0; i < rows; ++i)
vec1[i] = 10000;
NamesAndTypesListPtr names_and_types = std::make_shared<NamesAndTypesList>(); block.insert(column1);
names_and_types->push_back(NameAndTypePair("d", std::make_shared<DataTypeDate>()));
names_and_types->push_back(NameAndTypePair("a", std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt32>())));
ASTPtr primary_expr; ColumnWithTypeAndName column2;
column2.name = "a";
column2.type = table->getDataTypeByName("a");
column2.column = column2.type->createColumn();
for (size_t i = 0; i < rows; ++i)
column2.column->insert(Array((rand() % 10) == 0 ? (rand() % 10) : 0, i));
block.insert(column2);
BlockOutputStreamPtr out = table->write({}, {});
out->write(block);
}
/// читаем из неё
{
Names column_names;
column_names.push_back("d");
column_names.push_back("a");
QueryProcessingStage::Enum stage;
ASTPtr select;
Expected expected = ""; Expected expected = "";
String primary_expr_str = "d"; String select_str = "SELECT * FROM test";
const char * begin = primary_expr_str.data(); const char * begin = select_str.data();
const char * end = begin + primary_expr_str.size(); const char * end = begin + select_str.size();
const char * max_parsed_pos = begin; const char * max_parsed_pos = begin;
ParserExpressionList parser(false); ParserSelectQuery parser;
if (!parser.parse(begin, end, primary_expr, max_parsed_pos, expected)) if (!parser.parse(begin, end, select, max_parsed_pos, expected))
throw Poco::Exception("Cannot parse " + primary_expr_str); throw Poco::Exception("Cannot parse " + primary_expr_str);
MergeTreeData::MergingParams params; BlockInputStreamPtr in = table->read(column_names, select, context, Settings(), stage)[0];
params.mode = MergeTreeData::MergingParams::Ordinary;
StoragePtr table = StorageMergeTree::create( Block sample;
"./", "default", "test",
names_and_types, {}, {}, ColumnDefaults{},
context, primary_expr, "d",
nullptr, 101, params, {});
/// пишем в неё
{ {
Block block; ColumnWithTypeAndName col;
col.type = names_and_types->front().type;
ColumnWithTypeAndName column1; sample.insert(col);
column1.name = "d"; }
column1.type = table->getDataTypeByName("d"); {
column1.column = column1.type->createColumn(); ColumnWithTypeAndName col;
ColumnUInt16::Container_t & vec1 = typeid_cast<ColumnUInt16 &>(*column1.column).getData(); col.type = names_and_types->back().type;
sample.insert(col);
vec1.resize(rows);
for (size_t i = 0; i < rows; ++i)
vec1[i] = 10000;
block.insert(column1);
ColumnWithTypeAndName column2;
column2.name = "a";
column2.type = table->getDataTypeByName("a");
column2.column = column2.type->createColumn();
for (size_t i = 0; i < rows; ++i)
column2.column->insert(Array((rand() % 10) == 0 ? (rand() % 10) : 0, i));
block.insert(column2);
SharedPtr<IBlockOutputStream> out = table->write({}, {});
out->write(block);
} }
/// читаем из неё WriteBufferFromFileDescriptor out_buf(STDOUT_FILENO);
{
Names column_names;
column_names.push_back("d");
column_names.push_back("a");
QueryProcessingStage::Enum stage; RowOutputStreamPtr output_ = std::make_shared<TabSeparatedRowOutputStream>(out_buf, sample);
BlockOutputStreamFromRowOutputStream output(output_);
ASTPtr select; copyData(*in, output);
Expected expected = "";
String select_str = "SELECT * FROM test";
const char * begin = select_str.data();
const char * end = begin + select_str.size();
const char * max_parsed_pos = begin;
ParserSelectQuery parser;
if (!parser.parse(begin, end, select, max_parsed_pos, expected))
throw Poco::Exception("Cannot parse " + primary_expr_str);
SharedPtr<IBlockInputStream> in = table->read(column_names, select, context, Settings(), stage)[0];
Block sample;
{
ColumnWithTypeAndName col;
col.type = names_and_types->front().type;
sample.insert(col);
}
{
ColumnWithTypeAndName col;
col.type = names_and_types->back().type;
sample.insert(col);
}
WriteBufferFromFileDescriptor out_buf(STDOUT_FILENO);
RowOutputStreamPtr output_ = std::make_shared<TabSeparatedRowOutputStream>(out_buf, sample);
BlockOutputStreamFromRowOutputStream output(output_);
copyData(*in, output);
}
}
catch (const Exception & e)
{
std::cerr << e.displayText() << ", stack trace: \n\n" << e.getStackTrace().toString() << std::endl;
return 1;
} }
return 0; return 0;
} }
catch (const DB::Exception & e)
{
std::cerr << e.displayText() << ", stack trace: \n\n" << e.getStackTrace().toString() << std::endl;
throw;
}

View File

@ -54,7 +54,7 @@ try
block.insert(column2); block.insert(column2);
SharedPtr<IBlockOutputStream> out = table->write({}, {}); BlockOutputStreamPtr out = table->write({}, {});
out->write(block); out->write(block);
} }
@ -66,7 +66,7 @@ try
QueryProcessingStage::Enum stage; QueryProcessingStage::Enum stage;
SharedPtr<IBlockInputStream> in = table->read(column_names, 0, Context{}, Settings(), stage)[0]; BlockInputStreamPtr in = table->read(column_names, 0, Context{}, Settings(), stage)[0];
Block sample; Block sample;
{ {

View File

@ -263,7 +263,7 @@ StoragePtr TableFunctionRemote::execute(ASTPtr ast_function, Context & context)
if (names.empty()) if (names.empty())
throw Exception("Shard list is empty after parsing first argument", ErrorCodes::BAD_ARGUMENTS); throw Exception("Shard list is empty after parsing first argument", ErrorCodes::BAD_ARGUMENTS);
SharedPtr<Cluster> cluster = new Cluster(context.getSettings(), names, username, password); auto cluster = std::make_shared<Cluster>(context.getSettings(), names, username, password);
return StorageDistributed::create( return StorageDistributed::create(
getName(), getName(),

View File

@ -63,7 +63,7 @@ StoragePtr TableFunctionShardByHash::execute(ASTPtr ast_function, Context & cont
const Cluster & cluster = context.getCluster(cluster_name); const Cluster & cluster = context.getCluster(cluster_name);
size_t shard_index = sipHash64(key) % cluster.getShardCount(); size_t shard_index = sipHash64(key) % cluster.getShardCount();
SharedPtr<Cluster> shard = cluster.getClusterWithSingleShard(shard_index).release(); std::shared_ptr<Cluster> shard(cluster.getClusterWithSingleShard(shard_index).release());
return StorageDistributed::create( return StorageDistributed::create(
getName(), getName(),