mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 01:22:04 +00:00
Using std::shared_ptr for data types [#METR-21503].
This commit is contained in:
parent
3c1c9d46f7
commit
646d70a545
@ -116,7 +116,7 @@ public:
|
||||
UInt64 type;
|
||||
|
||||
Block block;
|
||||
SharedPtr<Exception> exception;
|
||||
std::unique_ptr<Exception> exception;
|
||||
Progress progress;
|
||||
BlockStreamProfileInfo profile_info;
|
||||
|
||||
@ -181,8 +181,8 @@ public:
|
||||
*/
|
||||
void fillBlockExtraInfo(BlockExtraInfo & info) const;
|
||||
|
||||
size_t outBytesCount() const { return !out.isNull() ? out->count() : 0; }
|
||||
size_t inBytesCount() const { return !in.isNull() ? in->count() : 0; }
|
||||
size_t outBytesCount() const { return out ? out->count() : 0; }
|
||||
size_t inBytesCount() const { return in ? in->count() : 0; }
|
||||
|
||||
private:
|
||||
String host;
|
||||
@ -210,8 +210,8 @@ private:
|
||||
UInt64 server_revision = 0;
|
||||
|
||||
Poco::Net::StreamSocket socket;
|
||||
SharedPtr<ReadBuffer> in;
|
||||
SharedPtr<WriteBuffer> out;
|
||||
std::shared_ptr<ReadBuffer> in;
|
||||
std::shared_ptr<WriteBuffer> out;
|
||||
|
||||
String query_id;
|
||||
UInt64 compression; /// Сжимать ли данные при взаимодействии с сервером.
|
||||
@ -229,11 +229,11 @@ private:
|
||||
Poco::Timespan ping_timeout;
|
||||
|
||||
/// Откуда читать результат выполнения запроса.
|
||||
SharedPtr<ReadBuffer> maybe_compressed_in;
|
||||
std::shared_ptr<ReadBuffer> maybe_compressed_in;
|
||||
BlockInputStreamPtr block_in;
|
||||
|
||||
/// Куда писать данные INSERT-а.
|
||||
SharedPtr<WriteBuffer> maybe_compressed_out;
|
||||
std::shared_ptr<WriteBuffer> maybe_compressed_out;
|
||||
BlockOutputStreamPtr block_out;
|
||||
|
||||
/// логгер, создаваемый лениво, чтобы не обращаться к DNS в конструкторе
|
||||
@ -266,7 +266,7 @@ private:
|
||||
bool ping();
|
||||
|
||||
Block receiveData();
|
||||
SharedPtr<Exception> receiveException();
|
||||
std::unique_ptr<Exception> receiveException();
|
||||
Progress receiveProgress();
|
||||
BlockStreamProfileInfo receiveProfileInfo();
|
||||
|
||||
|
@ -8,9 +8,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Интерфейс для пулов соединений.
|
||||
*
|
||||
* Использование (на примере обычного ConnectionPool):
|
||||
|
@ -2,8 +2,6 @@
|
||||
|
||||
#include <string.h> // memcpy
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Common/Exception.h>
|
||||
#include <DB/Common/Arena.h>
|
||||
|
||||
@ -22,8 +20,6 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
/** Cтолбeц значений типа массив.
|
||||
* В памяти он представлен, как один столбец вложенного типа, размер которого равен сумме размеров всех массивов,
|
||||
* и как массив смещений в нём, который позволяет достать каждый элемент.
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Core/Field.h>
|
||||
#include <DB/Common/Exception.h>
|
||||
#include <DB/Columns/ColumnVector.h>
|
||||
@ -21,8 +19,6 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
class IColumnConst : public IColumn
|
||||
{
|
||||
@ -57,11 +53,11 @@ namespace ColumnConstDetails
|
||||
|
||||
|
||||
/** Столбец-константа может содержать внутри себя само значение,
|
||||
* или, в случае массивов, SharedPtr от значения-массива,
|
||||
* или, в случае массивов, std::shared_ptr от значения-массива,
|
||||
* чтобы избежать проблем производительности при копировании очень больших массивов.
|
||||
*
|
||||
* T - тип значения,
|
||||
* DataHolder - как значение хранится в таблице (либо T, либо SharedPtr<T>)
|
||||
* DataHolder - как значение хранится в таблице (либо T, либо std::shared_ptr<T>)
|
||||
* Derived должен реализовать методы getDataFromHolderImpl - получить ссылку на значение из holder-а.
|
||||
*
|
||||
* Для строк и массивов реализации sizeOfField и byteSize могут быть некорректными.
|
||||
@ -232,10 +228,10 @@ public:
|
||||
|
||||
|
||||
template <>
|
||||
class ColumnConst<Array> final : public ColumnConstBase<Array, SharedPtr<Array>, ColumnConst<Array>>
|
||||
class ColumnConst<Array> final : public ColumnConstBase<Array, std::shared_ptr<Array>, ColumnConst<Array>>
|
||||
{
|
||||
private:
|
||||
friend class ColumnConstBase<Array, SharedPtr<Array>, ColumnConst<Array>>;
|
||||
friend class ColumnConstBase<Array, std::shared_ptr<Array>, ColumnConst<Array>>;
|
||||
|
||||
Array & getDataFromHolderImpl() { return *data; }
|
||||
const Array & getDataFromHolderImpl() const { return *data; }
|
||||
@ -243,10 +239,10 @@ private:
|
||||
public:
|
||||
/// data_type_ должен быть ненулевым.
|
||||
ColumnConst(size_t s_, const Array & data_, DataTypePtr data_type_ = DataTypePtr())
|
||||
: ColumnConstBase<Array, SharedPtr<Array>, ColumnConst<Array>>(s_, new Array(data_), data_type_) {}
|
||||
: ColumnConstBase<Array, std::shared_ptr<Array>, ColumnConst<Array>>(s_, std::make_shared<Array>(data_), data_type_) {}
|
||||
|
||||
ColumnConst(size_t s_, const SharedPtr<Array> & data_, DataTypePtr data_type_ = DataTypePtr())
|
||||
: ColumnConstBase<Array, SharedPtr<Array>, ColumnConst<Array>>(s_, data_, data_type_) {}
|
||||
ColumnConst(size_t s_, const std::shared_ptr<Array> & data_, DataTypePtr data_type_ = DataTypePtr())
|
||||
: ColumnConstBase<Array, std::shared_ptr<Array>, ColumnConst<Array>>(s_, data_, data_type_) {}
|
||||
|
||||
StringRef getDataAt(size_t n) const override;
|
||||
StringRef getDataAtWithTerminatingZero(size_t n) const override;
|
||||
@ -267,10 +263,10 @@ public:
|
||||
|
||||
|
||||
template <>
|
||||
class ColumnConst<Tuple> final : public ColumnConstBase<Tuple, SharedPtr<Tuple>, ColumnConst<Tuple>>
|
||||
class ColumnConst<Tuple> final : public ColumnConstBase<Tuple, std::shared_ptr<Tuple>, ColumnConst<Tuple>>
|
||||
{
|
||||
private:
|
||||
friend class ColumnConstBase<Tuple, SharedPtr<Tuple>, ColumnConst<Tuple>>;
|
||||
friend class ColumnConstBase<Tuple, std::shared_ptr<Tuple>, ColumnConst<Tuple>>;
|
||||
|
||||
Tuple & getDataFromHolderImpl() { return *data; }
|
||||
const Tuple & getDataFromHolderImpl() const { return *data; }
|
||||
@ -278,10 +274,10 @@ private:
|
||||
public:
|
||||
/// data_type_ должен быть ненулевым.
|
||||
ColumnConst(size_t s_, const Tuple & data_, DataTypePtr data_type_ = DataTypePtr())
|
||||
: ColumnConstBase<Tuple, SharedPtr<Tuple>, ColumnConst<Tuple>>(s_, new Tuple(data_), data_type_) {}
|
||||
: ColumnConstBase<Tuple, std::shared_ptr<Tuple>, ColumnConst<Tuple>>(s_, std::make_shared<Tuple>(data_), data_type_) {}
|
||||
|
||||
ColumnConst(size_t s_, const SharedPtr<Tuple> & data_, DataTypePtr data_type_ = DataTypePtr())
|
||||
: ColumnConstBase<Tuple, SharedPtr<Tuple>, ColumnConst<Tuple>>(s_, data_, data_type_) {}
|
||||
ColumnConst(size_t s_, const std::shared_ptr<Tuple> & data_, DataTypePtr data_type_ = DataTypePtr())
|
||||
: ColumnConstBase<Tuple, std::shared_ptr<Tuple>, ColumnConst<Tuple>>(s_, data_, data_type_) {}
|
||||
|
||||
StringRef getDataAt(size_t n) const override;
|
||||
StringRef getDataAtWithTerminatingZero(size_t n) const override;
|
||||
|
@ -2,9 +2,9 @@
|
||||
|
||||
#include <cerrno>
|
||||
#include <vector>
|
||||
#include <memory>
|
||||
|
||||
#include <Poco/Exception.h>
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Common/StackTrace.h>
|
||||
|
||||
@ -90,7 +90,7 @@ std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace);
|
||||
|
||||
void rethrowFirstException(Exceptions & exceptions);
|
||||
|
||||
Poco::SharedPtr<Poco::Exception> convertCurrentException();
|
||||
std::unique_ptr<Poco::Exception> convertCurrentException();
|
||||
|
||||
|
||||
template <typename T>
|
||||
|
@ -4,8 +4,6 @@
|
||||
#include <vector>
|
||||
#include <memory>
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <re2/re2.h>
|
||||
#include <re2_st/re2.h>
|
||||
|
||||
|
@ -84,7 +84,7 @@ public:
|
||||
Object & operator*() & { return *data->data.object; }
|
||||
const Object & operator*() const & { return *data->data.object; }
|
||||
|
||||
bool isNull() const { return data.isNull(); }
|
||||
bool isNull() const { return data == nullptr; }
|
||||
|
||||
private:
|
||||
std::shared_ptr<PoolEntryHelper> data;
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <vector>
|
||||
#include <algorithm>
|
||||
#include <type_traits>
|
||||
#include <functional>
|
||||
|
||||
@ -27,8 +28,6 @@ using TupleBackend = std::vector<Field>;
|
||||
STRONG_TYPEDEF(TupleBackend, Tuple); /// Значение типа "кортеж"
|
||||
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
/** 32 хватает с запасом (достаточно 28), но выбрано круглое число,
|
||||
* чтобы арифметика при использовании массивов из Field была проще (не содержала умножения).
|
||||
*/
|
||||
|
@ -18,12 +18,12 @@ struct SortColumnDescription
|
||||
String column_name; /// Имя столбца.
|
||||
size_t column_number; /// Номер столбца (используется, если не задано имя).
|
||||
int direction; /// 1 - по возрастанию, -1 - по убыванию.
|
||||
Poco::SharedPtr<Collator> collator; /// Collator для locale-specific сортировки строк
|
||||
std::shared_ptr<Collator> collator; /// Collator для locale-specific сортировки строк
|
||||
|
||||
SortColumnDescription(size_t column_number_, int direction_, const Poco::SharedPtr<Collator> & collator_ = nullptr)
|
||||
SortColumnDescription(size_t column_number_, int direction_, const std::shared_ptr<Collator> & collator_ = nullptr)
|
||||
: column_number(column_number_), direction(direction_), collator(collator_) {}
|
||||
|
||||
SortColumnDescription(String column_name_, int direction_, const Poco::SharedPtr<Collator> & collator_ = nullptr)
|
||||
SortColumnDescription(String column_name_, int direction_, const std::shared_ptr<Collator> & collator_ = nullptr)
|
||||
: column_name(column_name_), column_number(0), direction(direction_), collator(collator_) {}
|
||||
|
||||
/// Для IBlockInputStream.
|
||||
@ -31,7 +31,7 @@ struct SortColumnDescription
|
||||
{
|
||||
std::stringstream res;
|
||||
res << column_name << ", " << column_number << ", " << direction;
|
||||
if (!collator.isNull())
|
||||
if (collator)
|
||||
res << ", collation locale: " << collator->getLocale();
|
||||
return res.str();
|
||||
}
|
||||
@ -96,7 +96,7 @@ struct SortCursorImpl
|
||||
|
||||
sort_columns.push_back(&*block.getByPosition(column_number).column);
|
||||
|
||||
need_collation[j] = !desc[j].collator.isNull() && sort_columns.back()->getName() == "ColumnString";
|
||||
need_collation[j] = desc[j].collator != nullptr && sort_columns.back()->getName() == "ColumnString";
|
||||
has_collation |= need_collation[j];
|
||||
}
|
||||
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
#include <DB/Core/ColumnWithTypeAndName.h>
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||
#include <DB/Interpreters/evaluateMissingDefaults.h>
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
|
||||
|
@ -11,8 +11,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Агрегирует поток блоков, используя заданные столбцы-ключи и агрегатные функции.
|
||||
* Столбцы с агрегатными функциями добавляет в конец блока.
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Core/Block.h>
|
||||
#include <DB/IO/WriteBuffer.h>
|
||||
#include <DB/DataStreams/IRowOutputStream.h>
|
||||
|
@ -6,8 +6,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Объединяет несколько источников в один.
|
||||
* В отличие от UnionBlockInputStream, делает это последовательно.
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||
|
||||
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Core/Block.h>
|
||||
#include <DB/IO/WriteBuffer.h>
|
||||
#include <DB/IO/WriteBufferValidUTF8.h>
|
||||
|
@ -1,15 +1,11 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Реализует реляционную операцию LIMIT.
|
||||
*/
|
||||
|
@ -7,8 +7,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Доагрегирует поток блоков, в котором каждый блок уже агрегирован.
|
||||
* Агрегатные функции в блоках не должны быть финализированы, чтобы их состояния можно было объединить.
|
||||
|
@ -14,8 +14,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Агрегирует несколько источников параллельно.
|
||||
* Производит агрегацию блоков из разных источников независимо в разных потоках, затем объединяет результаты.
|
||||
@ -264,7 +262,7 @@ private:
|
||||
Stopwatch watch;
|
||||
|
||||
for (auto & elem : many_data)
|
||||
elem = new AggregatedDataVariants;
|
||||
elem = std::make_shared<AggregatedDataVariants>();
|
||||
|
||||
processor.process();
|
||||
processor.wait();
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
|
||||
@ -10,7 +8,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
/** Удаляет из блока указанные столбцы.
|
||||
*/
|
||||
class RemoveColumnsBlockInputStream : public IProfilingBlockInputStream
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Core/Block.h>
|
||||
#include <DB/IO/WriteBuffer.h>
|
||||
#include <DB/DataStreams/IRowOutputStream.h>
|
||||
|
@ -10,8 +10,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Core/Block.h>
|
||||
#include <DB/Core/Names.h>
|
||||
#include <DB/IO/WriteBuffer.h>
|
||||
@ -11,9 +9,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Поток для вывода данных в формате "каждое значение на своей строке".
|
||||
*/
|
||||
class VerticalRowOutputStream : public IRowOutputStream
|
||||
|
@ -7,8 +7,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
class DataTypeArray final : public IDataType
|
||||
{
|
||||
|
@ -2,17 +2,12 @@
|
||||
|
||||
#include <ostream>
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/DataTypes/IDataType.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
class DataTypeString final : public IDataType
|
||||
{
|
||||
public:
|
||||
|
@ -185,10 +185,10 @@ public:
|
||||
using base_type = FunctionTransformWithDictionary;
|
||||
|
||||
private:
|
||||
const SharedPtr<typename DictGetter::Src> owned_dict;
|
||||
const std::shared_ptr<typename DictGetter::Src> owned_dict;
|
||||
|
||||
public:
|
||||
FunctionTransformWithDictionary(const SharedPtr<typename DictGetter::Src> & owned_dict_)
|
||||
FunctionTransformWithDictionary(const std::shared_ptr<typename DictGetter::Src> & owned_dict_)
|
||||
: owned_dict(owned_dict_)
|
||||
{
|
||||
if (!owned_dict)
|
||||
@ -278,10 +278,10 @@ public:
|
||||
using base_type = FunctionIsInWithDictionary;
|
||||
|
||||
private:
|
||||
const SharedPtr<typename DictGetter::Src> owned_dict;
|
||||
const std::shared_ptr<typename DictGetter::Src> owned_dict;
|
||||
|
||||
public:
|
||||
FunctionIsInWithDictionary(const SharedPtr<typename DictGetter::Src> & owned_dict_)
|
||||
FunctionIsInWithDictionary(const std::shared_ptr<typename DictGetter::Src> & owned_dict_)
|
||||
: owned_dict(owned_dict_)
|
||||
{
|
||||
if (!owned_dict)
|
||||
@ -411,10 +411,10 @@ public:
|
||||
using base_type = FunctionHierarchyWithDictionary;
|
||||
|
||||
private:
|
||||
const SharedPtr<typename DictGetter::Src> owned_dict;
|
||||
const std::shared_ptr<typename DictGetter::Src> owned_dict;
|
||||
|
||||
public:
|
||||
FunctionHierarchyWithDictionary(const SharedPtr<typename DictGetter::Src> & owned_dict_)
|
||||
FunctionHierarchyWithDictionary(const std::shared_ptr<typename DictGetter::Src> & owned_dict_)
|
||||
: owned_dict(owned_dict_)
|
||||
{
|
||||
if (!owned_dict)
|
||||
@ -684,10 +684,10 @@ public:
|
||||
}
|
||||
|
||||
private:
|
||||
const SharedPtr<RegionsNames> owned_dict;
|
||||
const std::shared_ptr<RegionsNames> owned_dict;
|
||||
|
||||
public:
|
||||
FunctionRegionToName(const SharedPtr<RegionsNames> & owned_dict_)
|
||||
FunctionRegionToName(const std::shared_ptr<RegionsNames> & owned_dict_)
|
||||
: owned_dict(owned_dict_)
|
||||
{
|
||||
if (!owned_dict)
|
||||
|
@ -4,7 +4,6 @@
|
||||
|
||||
#include <vector>
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
#include <Poco/Net/NetException.h>
|
||||
|
||||
#include <threadpool.hpp>
|
||||
@ -15,8 +14,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Записывает данные асинхронно с помощью двойной буферизации.
|
||||
*/
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <DB/IO/WriteBuffer.h>
|
||||
#include <DB/IO/HashingWriteBuffer.h>
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
#include <Poco/Net/HTTPClientSession.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -15,8 +15,8 @@ namespace DB
|
||||
class RemoteReadBuffer : public ReadBuffer
|
||||
{
|
||||
private:
|
||||
Poco::SharedPtr<ReadBufferFromHTTP> impl;
|
||||
|
||||
std::unique_ptr<ReadBufferFromHTTP> impl;
|
||||
|
||||
public:
|
||||
RemoteReadBuffer(
|
||||
const std::string & host,
|
||||
@ -34,7 +34,7 @@ public:
|
||||
std::make_pair("path", path),
|
||||
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()
|
||||
|
@ -752,7 +752,7 @@ struct AggregatedDataVariants : private boost::noncopyable
|
||||
M(serialized_two_level)
|
||||
};
|
||||
|
||||
using AggregatedDataVariantsPtr = SharedPtr<AggregatedDataVariants>;
|
||||
using AggregatedDataVariantsPtr = std::shared_ptr<AggregatedDataVariants>;
|
||||
using ManyAggregatedDataVariants = std::vector<AggregatedDataVariantsPtr>;
|
||||
|
||||
/** Как считаются "тотальные" значения при наличии WITH TOTALS?
|
||||
|
@ -278,7 +278,7 @@ public:
|
||||
void resetCaches() const;
|
||||
|
||||
const Cluster & getCluster(const std::string & cluster_name) const;
|
||||
Poco::SharedPtr<Clusters> getClusters() const;
|
||||
std::shared_ptr<Clusters> getClusters() const;
|
||||
|
||||
Compiler & getCompiler();
|
||||
QueryLog & getQueryLog();
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
#include <DB/Core/Field.h>
|
||||
|
||||
|
||||
@ -13,6 +14,6 @@ class Context;
|
||||
* Используется в редких случаях - для элемента множества в IN, для данных для INSERT.
|
||||
* Весьма неоптимально.
|
||||
*/
|
||||
Field evaluateConstantExpression(SharedPtr<IAST> & node, const Context & context);
|
||||
Field evaluateConstantExpression(Poco::SharedPtr<IAST> & node, const Context & context);
|
||||
|
||||
}
|
||||
|
@ -6,9 +6,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Список выражений типа "a, b + c, f(d)"
|
||||
*/
|
||||
class ASTExpressionList : public IAST
|
||||
@ -16,7 +13,7 @@ class ASTExpressionList : public IAST
|
||||
public:
|
||||
ASTExpressionList() = default;
|
||||
ASTExpressionList(const StringRange range_) : IAST(range_) {}
|
||||
|
||||
|
||||
/** Получить текст, который идентифицирует этот элемент. */
|
||||
String getID() const override { return "ExpressionList"; }
|
||||
|
||||
@ -25,7 +22,7 @@ public:
|
||||
const auto res = new ASTExpressionList(*this);
|
||||
ASTPtr ptr{res};
|
||||
res->children.clear();
|
||||
|
||||
|
||||
for (const auto & child : children)
|
||||
res->children.emplace_back(child->clone());
|
||||
|
||||
|
@ -6,25 +6,22 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Элемент выражения, после которого стоит ASC или DESC
|
||||
*/
|
||||
class ASTOrderByElement : public IAST
|
||||
{
|
||||
public:
|
||||
int direction; /// 1, если ASC, -1, если DESC
|
||||
|
||||
|
||||
/** Collator для locale-specific сортировки строк.
|
||||
* Если nullptr, то производится сортировка по байтам.
|
||||
*/
|
||||
Poco::SharedPtr<Collator> collator;
|
||||
|
||||
std::shared_ptr<Collator> collator;
|
||||
|
||||
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_) {}
|
||||
|
||||
|
||||
/** Получить текст, который идентифицирует этот элемент. */
|
||||
String getID() const override { return "OrderByElement"; }
|
||||
|
||||
@ -34,8 +31,11 @@ protected:
|
||||
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
|
||||
{
|
||||
children.front()->formatImpl(settings, state, frame);
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << (direction == -1 ? " DESC" : " ASC") << (settings.hilite ? hilite_none : "");
|
||||
if (!collator.isNull())
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "")
|
||||
<< (direction == -1 ? " DESC" : " ASC")
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
|
||||
if (collator)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " COLLATE " << (settings.hilite ? hilite_none : "")
|
||||
<< "'" << collator->getLocale() << "'";
|
||||
|
@ -8,8 +8,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Если прямо сейчас не s, то ошибка.
|
||||
* Если word_boundary установлен в true, и последний символ строки - словарный (\w),
|
||||
|
@ -11,8 +11,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
using Expected = const char *;
|
||||
|
||||
|
||||
|
@ -8,7 +8,6 @@
|
||||
#include <mutex>
|
||||
#include <Poco/RWLock.h>
|
||||
#include <Poco/Event.h>
|
||||
#include <Poco/SharedPtr.h>
|
||||
#include <DB/Core/Types.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -368,8 +368,9 @@ private:
|
||||
|
||||
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_stream = new HashingWriteBuffer(*index_file_stream);
|
||||
index_file_stream = std::make_unique<WriteBufferFromFile>(
|
||||
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;
|
||||
|
||||
SharedPtr<WriteBufferFromFile> index_file_stream;
|
||||
SharedPtr<HashingWriteBuffer> index_stream;
|
||||
std::unique_ptr<WriteBufferFromFile> index_file_stream;
|
||||
std::unique_ptr<HashingWriteBuffer> index_stream;
|
||||
MergeTreeData::DataPart::Index index_columns;
|
||||
};
|
||||
|
||||
using MergedBlockOutputStreamPtr = Poco::SharedPtr<MergedBlockOutputStream>;
|
||||
|
||||
/// Записывает только те, столбцы, что лежат в block
|
||||
class MergedColumnOnlyOutputStream : public IMergedBlockOutputStream
|
||||
|
@ -8,7 +8,6 @@
|
||||
#include <zkutil/SingleBarrier.h>
|
||||
|
||||
#include <Poco/Util/LayeredConfiguration.h>
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <string>
|
||||
#include <thread>
|
||||
|
@ -42,7 +42,7 @@ public:
|
||||
NamesAndTypesListPtr columns_, /// Список столбцов.
|
||||
const String & remote_database_, /// БД на удалённых серверах.
|
||||
const String & remote_table_, /// Имя таблицы на удалённых серверах.
|
||||
SharedPtr<Cluster> & owned_cluster_,
|
||||
std::shared_ptr<Cluster> & owned_cluster_,
|
||||
Context & context_);
|
||||
|
||||
std::string getName() const override { return "Distributed"; }
|
||||
@ -134,8 +134,9 @@ private:
|
||||
Context & context;
|
||||
Logger * log = &Logger::get("StorageDistributed");
|
||||
|
||||
/// Используется только, если таблица должна владеть объектом Cluster, которым больше никто не владеет - для реализации TableFunctionRemote.
|
||||
SharedPtr<Cluster> owned_cluster;
|
||||
/// Используется только, если таблица должна владеть объектом Cluster,
|
||||
/// которым больше никто не владеет - для реализации TableFunctionRemote.
|
||||
std::shared_ptr<Cluster> owned_cluster;
|
||||
|
||||
/// Соединения с удалёнными серверами.
|
||||
const Cluster & cluster;
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
@ -9,9 +7,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Реализует системную таблицу databases, которая позволяет получить информацию о всех БД.
|
||||
*/
|
||||
class StorageSystemDatabases : public IStorage
|
||||
|
@ -1,16 +1,11 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Реализует хранилище для системной таблицы Numbers.
|
||||
* Таблица содержит единственный столбец number UInt64.
|
||||
* Из этой таблицы можно прочитать все натуральные числа, начиная с 0 (до 2^64 - 1, а потом заново).
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
@ -9,9 +7,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Реализует системную таблицу tables, которая позволяет получить информацию о всех таблицах.
|
||||
*/
|
||||
class StorageSystemParts : public IStorage
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
@ -9,9 +7,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Реализует системную таблицу processes, которая позволяет получить информацию о запросах, исполняющихся в данный момент.
|
||||
*/
|
||||
class StorageSystemProcesses : public IStorage
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
@ -9,9 +7,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Реализует системную таблицу replicas, которая позволяет получить информацию о статусе реплицируемых таблиц.
|
||||
*/
|
||||
class StorageSystemReplicas : public IStorage
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
@ -9,9 +7,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Реализует системную таблицу replication_queue, которая позволяет посмотреть очереди репликации для реплицируемых таблиц.
|
||||
*/
|
||||
class StorageSystemReplicationQueue : public IStorage
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
@ -9,9 +7,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Реализует системную таблицу settings, которая позволяет получить информацию о текущих настройках.
|
||||
*/
|
||||
class StorageSystemSettings : public IStorage
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
@ -9,9 +7,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Реализует системную таблицу tables, которая позволяет получить информацию о всех таблицах.
|
||||
*/
|
||||
class StorageSystemTables : public IStorage
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
@ -9,9 +7,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Реализует системную таблицу zookeeper, которая позволяет просматривать данные в ZooKeeper в целях отладки.
|
||||
*/
|
||||
class StorageSystemZooKeeper : public IStorage
|
||||
|
@ -10,7 +10,6 @@
|
||||
#include <random>
|
||||
|
||||
#include <Poco/File.h>
|
||||
#include <Poco/SharedPtr.h>
|
||||
#include <Poco/Util/Application.h>
|
||||
|
||||
#include <DB/Common/Stopwatch.h>
|
||||
|
@ -13,7 +13,6 @@
|
||||
#include <boost/program_options.hpp>
|
||||
|
||||
#include <Poco/File.h>
|
||||
#include <Poco/SharedPtr.h>
|
||||
#include <Poco/Util/Application.h>
|
||||
|
||||
#include <common/ClickHouseRevision.h>
|
||||
@ -114,8 +113,6 @@ namespace ErrorCodes
|
||||
extern const int CLIENT_OUTPUT_FORMAT_SPECIFIED;
|
||||
}
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
class Client : public Poco::Util::Application
|
||||
{
|
||||
@ -172,7 +169,7 @@ private:
|
||||
ASTPtr parsed_query;
|
||||
|
||||
/// Последнее полученное от сервера исключение. Для кода возврата в неинтерактивном режиме.
|
||||
Poco::SharedPtr<DB::Exception> last_exception;
|
||||
std::unique_ptr<Exception> last_exception;
|
||||
|
||||
/// Было ли в последнем запросе исключение.
|
||||
bool got_exception = false;
|
||||
@ -881,7 +878,7 @@ private:
|
||||
|
||||
case Protocol::Server::Exception:
|
||||
onException(*packet.exception);
|
||||
last_exception = packet.exception;
|
||||
last_exception = std::move(packet.exception);
|
||||
return false;
|
||||
|
||||
case Protocol::Server::EndOfStream:
|
||||
@ -908,7 +905,7 @@ private:
|
||||
|
||||
case Protocol::Server::Exception:
|
||||
onException(*packet.exception);
|
||||
last_exception = packet.exception;
|
||||
last_exception = std::move(packet.exception);
|
||||
return false;
|
||||
|
||||
default:
|
||||
|
@ -51,8 +51,8 @@ void Connection::connect()
|
||||
socket.setSendTimeout(send_timeout);
|
||||
socket.setNoDelay(true);
|
||||
|
||||
in = new ReadBufferFromPocoSocket(socket);
|
||||
out = new WriteBufferFromPocoSocket(socket);
|
||||
in = std::make_shared<ReadBufferFromPocoSocket>(socket);
|
||||
out = std::make_shared<WriteBufferFromPocoSocket>(socket);
|
||||
|
||||
connected = true;
|
||||
|
||||
@ -275,10 +275,10 @@ void Connection::sendQuery(const String & query, const String & query_id_, UInt6
|
||||
|
||||
writeStringBinary(query, *out);
|
||||
|
||||
maybe_compressed_in = nullptr;
|
||||
maybe_compressed_out = nullptr;
|
||||
block_in = nullptr;
|
||||
block_out = nullptr;
|
||||
maybe_compressed_in.reset();
|
||||
maybe_compressed_out.reset();
|
||||
block_in.reset();
|
||||
block_out.reset();
|
||||
|
||||
/// Если версия сервера достаточно новая и стоит флаг, отправляем пустой блок, символизируя конец передачи данных.
|
||||
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 (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
|
||||
maybe_compressed_out = out;
|
||||
|
||||
@ -502,7 +502,7 @@ void Connection::initBlockInput()
|
||||
if (!block_in)
|
||||
{
|
||||
if (compression == Protocol::Compression::Enable)
|
||||
maybe_compressed_in = new CompressedReadBuffer(*in);
|
||||
maybe_compressed_in = std::make_shared<CompressedReadBuffer>(*in);
|
||||
else
|
||||
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");
|
||||
|
||||
Exception e;
|
||||
readException(e, *in, "Received from " + getDescription());
|
||||
return e.clone();
|
||||
return { e.clone() };
|
||||
}
|
||||
|
||||
|
||||
|
@ -114,7 +114,7 @@ std::string getCurrentExceptionMessage(bool with_stacktrace)
|
||||
}
|
||||
|
||||
|
||||
Poco::SharedPtr<Poco::Exception> convertCurrentException()
|
||||
std::unique_ptr<Poco::Exception> convertCurrentException()
|
||||
{
|
||||
try
|
||||
{
|
||||
@ -122,19 +122,19 @@ Poco::SharedPtr<Poco::Exception> convertCurrentException()
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
return e.clone();
|
||||
return std::unique_ptr<Poco::Exception>{ e.clone() };
|
||||
}
|
||||
catch (const Poco::Exception & e)
|
||||
{
|
||||
return e.clone();
|
||||
return std::unique_ptr<Poco::Exception>{ e.clone() };
|
||||
}
|
||||
catch (const std::exception & e)
|
||||
{
|
||||
return new Exception(e.what(), ErrorCodes::STD_EXCEPTION);
|
||||
return std::make_unique<Exception>(e.what(), ErrorCodes::STD_EXCEPTION);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
return new Exception("Unknown exception", ErrorCodes::UNKNOWN_EXCEPTION);
|
||||
return std::make_unique<Exception>("Unknown exception", ErrorCodes::UNKNOWN_EXCEPTION);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -12,7 +12,7 @@ Block AggregatingBlockInputStream::readImpl()
|
||||
if (!executed)
|
||||
{
|
||||
executed = true;
|
||||
AggregatedDataVariantsPtr data_variants = new AggregatedDataVariants;
|
||||
AggregatedDataVariantsPtr data_variants = std::make_shared<AggregatedDataVariants>();
|
||||
|
||||
Aggregator::CancellationHook hook = [&]() { return this->isCancelled(); };
|
||||
aggregator.setCancellationHook(hook);
|
||||
|
@ -6,9 +6,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
JSONCompactRowOutputStream::JSONCompactRowOutputStream(WriteBuffer & ostr_, const Block & sample_)
|
||||
: JSONRowOutputStream(ostr_, sample_)
|
||||
{
|
||||
|
@ -6,8 +6,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
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_)
|
||||
{
|
||||
@ -19,7 +17,7 @@ Block LimitBlockInputStream::readImpl()
|
||||
{
|
||||
Block res;
|
||||
size_t rows = 0;
|
||||
|
||||
|
||||
/// pos - сколько строк было прочитано, включая последний прочитанный блок
|
||||
|
||||
if (pos >= offset + limit)
|
||||
|
@ -6,9 +6,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
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_)
|
||||
{
|
||||
|
@ -7,9 +7,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
VerticalRowOutputStream::VerticalRowOutputStream(WriteBuffer & ostr_, const Block & sample_)
|
||||
: ostr(ostr_), sample(sample_), field_number(0), row_number(0)
|
||||
{
|
||||
|
@ -6,9 +6,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
XMLRowOutputStream::XMLRowOutputStream(WriteBuffer & ostr_, const Block & sample_)
|
||||
: dst_ostr(ostr_)
|
||||
{
|
||||
|
@ -15,81 +15,76 @@
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
try
|
||||
{
|
||||
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);
|
||||
Logger::root().setChannel(channel);
|
||||
Logger::root().setLevel("trace");
|
||||
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));
|
||||
block1.insert(column1);
|
||||
|
||||
Block block1;
|
||||
|
||||
{
|
||||
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));
|
||||
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);
|
||||
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);
|
||||
}
|
||||
catch (const Exception & e)
|
||||
|
||||
Block block2;
|
||||
|
||||
{
|
||||
std::cerr << e.what() << ", " << e.displayText() << std::endl;
|
||||
return 1;
|
||||
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);
|
||||
|
||||
return 0;
|
||||
}
|
||||
catch (const DB::Exception & e)
|
||||
{
|
||||
std::cerr << e.what() << ", " << e.displayText() << std::endl;
|
||||
throw;
|
||||
}
|
||||
|
@ -20,65 +20,61 @@
|
||||
#include <DB/Interpreters/ExpressionActions.h>
|
||||
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
try
|
||||
{
|
||||
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;
|
||||
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;
|
||||
|
||||
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;
|
||||
stopwatch.stop();
|
||||
std::cout << std::fixed << std::setprecision(2)
|
||||
<< "Elapsed " << stopwatch.elapsedSeconds() << " sec."
|
||||
<< ", " << n / stopwatch.elapsedSeconds() << " rows/sec."
|
||||
<< std::endl;
|
||||
}
|
||||
|
||||
return 0;
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
std::cerr << e.what() << ", " << e.displayText() << std::endl;
|
||||
throw;
|
||||
}
|
||||
|
||||
|
@ -22,69 +22,64 @@
|
||||
#include <DB/Interpreters/ExpressionActions.h>
|
||||
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
try
|
||||
{
|
||||
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;
|
||||
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;
|
||||
|
||||
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;
|
||||
stopwatch.stop();
|
||||
std::cout << std::fixed << std::setprecision(2)
|
||||
<< "Elapsed " << stopwatch.elapsedSeconds() << " sec."
|
||||
<< ", " << n / stopwatch.elapsedSeconds() << " rows/sec."
|
||||
<< std::endl;
|
||||
}
|
||||
|
||||
return 0;
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
std::cerr << e.what() << ", " << e.displayText() << std::endl;
|
||||
throw;
|
||||
}
|
||||
|
@ -125,7 +125,7 @@ int main(int argc, char ** argv)
|
||||
|
||||
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<LimitBlockInputStream>(in, 10, 0);
|
||||
|
||||
|
@ -23,9 +23,6 @@
|
||||
#include <DB/Interpreters/ExpressionActions.h>
|
||||
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
void thread1(DB::BlockInputStreamPtr in, DB::BlockOutputStreamPtr out, DB::WriteBuffer & out_buf)
|
||||
{
|
||||
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)
|
||||
try
|
||||
{
|
||||
using namespace DB;
|
||||
|
||||
try
|
||||
{
|
||||
std::string input = "SELECT number, number % 10000000 == 1";
|
||||
std::string input = "SELECT number, number % 10000000 == 1";
|
||||
|
||||
ParserSelectQuery parser;
|
||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "");
|
||||
ParserSelectQuery parser;
|
||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "");
|
||||
|
||||
formatAST(*ast, std::cerr);
|
||||
std::cerr << std::endl;
|
||||
formatAST(*ast, std::cerr);
|
||||
std::cerr << std::endl;
|
||||
|
||||
Context context;
|
||||
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();
|
||||
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");
|
||||
StoragePtr table = StorageSystemNumbers::create("Numbers");
|
||||
|
||||
Names column_names;
|
||||
column_names.push_back("number");
|
||||
Names column_names;
|
||||
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 in2 = fork.createInput();
|
||||
BlockInputStreamPtr in1 = fork.createInput();
|
||||
BlockInputStreamPtr in2 = fork.createInput();
|
||||
|
||||
in1 = std::make_shared<FilterBlockInputStream>(in1, expression, 1);
|
||||
in1 = std::make_shared<LimitBlockInputStream>(in1, 10, 0);
|
||||
in1 = std::make_shared<FilterBlockInputStream>(in1, expression, 1);
|
||||
in1 = std::make_shared<LimitBlockInputStream>(in1, 10, 0);
|
||||
|
||||
in2 = std::make_shared<FilterBlockInputStream>(in2, expression, 1);
|
||||
in2 = std::make_shared<LimitBlockInputStream>(in2, 20, 5);
|
||||
in2 = std::make_shared<FilterBlockInputStream>(in2, expression, 1);
|
||||
in2 = std::make_shared<LimitBlockInputStream>(in2, 20, 5);
|
||||
|
||||
Block out_sample = expression->getSampleBlock();
|
||||
Block out_sample = expression->getSampleBlock();
|
||||
|
||||
WriteBufferFromOStream ob1(std::cout);
|
||||
WriteBufferFromOStream ob2(std::cerr);
|
||||
WriteBufferFromOStream ob1(std::cout);
|
||||
WriteBufferFromOStream ob2(std::cerr);
|
||||
|
||||
BlockOutputStreamPtr out1 = context.getOutputFormat("TabSeparated", ob1, out_sample);
|
||||
BlockOutputStreamPtr out2 = context.getOutputFormat("TabSeparated", ob2, out_sample);
|
||||
BlockOutputStreamPtr out1 = context.getOutputFormat("TabSeparated", ob1, out_sample);
|
||||
BlockOutputStreamPtr out2 = context.getOutputFormat("TabSeparated", ob2, out_sample);
|
||||
|
||||
std::thread thr1(std::bind(thread1, in1, out1, std::ref(ob1)));
|
||||
std::thread thr2(std::bind(thread2, in2, out2, std::ref(ob2)));
|
||||
std::thread thr1(std::bind(thread1, in1, out1, std::ref(ob1)));
|
||||
std::thread thr2(std::bind(thread2, in2, out2, std::ref(ob2)));
|
||||
|
||||
fork.run();
|
||||
fork.run();
|
||||
|
||||
thr1.join();
|
||||
thr2.join();
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
std::cerr << e.what() << ", " << e.displayText() << std::endl;
|
||||
return 1;
|
||||
}
|
||||
thr1.join();
|
||||
thr2.join();
|
||||
|
||||
return 0;
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
std::cerr << e.what() << ", " << e.displayText() << std::endl;
|
||||
throw;
|
||||
}
|
||||
|
@ -11,7 +11,6 @@
|
||||
#include <DB/DataStreams/glueBlockInputStreams.h>
|
||||
|
||||
|
||||
using Poco::SharedPtr;
|
||||
using namespace DB;
|
||||
|
||||
|
||||
@ -33,69 +32,66 @@ void forkThread(ForkPtr fork)
|
||||
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
try
|
||||
{
|
||||
try
|
||||
{
|
||||
Context context;
|
||||
Context context;
|
||||
|
||||
context.setGlobalContext(context);
|
||||
context.setPath("./");
|
||||
context.setGlobalContext(context);
|
||||
context.setPath("./");
|
||||
|
||||
loadMetadata(context);
|
||||
loadMetadata(context);
|
||||
|
||||
context.setCurrentDatabase("default");
|
||||
context.setSetting("max_threads", 1UL);
|
||||
context.setCurrentDatabase("default");
|
||||
context.setSetting("max_threads", 1UL);
|
||||
|
||||
BlockIO io1 = executeQuery(
|
||||
"SELECT SearchPhrase, count()"
|
||||
" FROM hits"
|
||||
" WHERE SearchPhrase != ''"
|
||||
" GROUP BY SearchPhrase"
|
||||
" ORDER BY count() DESC"
|
||||
" LIMIT 10",
|
||||
context, QueryProcessingStage::Complete);
|
||||
BlockIO io1 = executeQuery(
|
||||
"SELECT SearchPhrase, count()"
|
||||
" FROM hits"
|
||||
" WHERE SearchPhrase != ''"
|
||||
" GROUP BY SearchPhrase"
|
||||
" ORDER BY count() DESC"
|
||||
" LIMIT 10",
|
||||
context, QueryProcessingStage::Complete);
|
||||
|
||||
BlockIO io2 = executeQuery(
|
||||
"SELECT count()"
|
||||
" FROM hits"
|
||||
" WHERE SearchPhrase != ''",
|
||||
context, QueryProcessingStage::Complete);
|
||||
BlockIO io2 = executeQuery(
|
||||
"SELECT count()"
|
||||
" FROM hits"
|
||||
" WHERE SearchPhrase != ''",
|
||||
context, QueryProcessingStage::Complete);
|
||||
|
||||
WriteBufferFromFileDescriptor wb(STDOUT_FILENO);
|
||||
WriteBufferFromFileDescriptor wb(STDOUT_FILENO);
|
||||
|
||||
BlockOutputStreamPtr out1 = context.getOutputFormat("TabSeparated", wb, io1.in_sample);
|
||||
BlockOutputStreamPtr out2 = context.getOutputFormat("TabSeparated", wb, io2.in_sample);
|
||||
BlockOutputStreamPtr out1 = context.getOutputFormat("TabSeparated", wb, io1.in_sample);
|
||||
BlockOutputStreamPtr out2 = context.getOutputFormat("TabSeparated", wb, io2.in_sample);
|
||||
|
||||
BlockInputStreams inputs;
|
||||
inputs.push_back(io1.in);
|
||||
inputs.push_back(io2.in);
|
||||
BlockInputStreams inputs;
|
||||
inputs.push_back(io1.in);
|
||||
inputs.push_back(io2.in);
|
||||
|
||||
for (size_t i = 0; i < inputs.size(); ++i)
|
||||
std::cerr << inputs[i]->getID() << std::endl;
|
||||
for (size_t i = 0; i < inputs.size(); ++i)
|
||||
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[1], out2, 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)));
|
||||
|
||||
for (size_t i = 0; i < forks.size(); ++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;
|
||||
}
|
||||
for (size_t i = 0; i < forks.size(); ++i)
|
||||
pool.schedule(std::bind(forkThread, forks[i]));
|
||||
|
||||
pool.wait();
|
||||
return 0;
|
||||
}
|
||||
catch (const DB::Exception & e)
|
||||
{
|
||||
std::cerr << e.what() << ", " << e.displayText() << std::endl;
|
||||
throw;
|
||||
}
|
||||
|
@ -101,7 +101,7 @@ try
|
||||
if (argc == 2 && 0 == strcmp(argv[1], "read"))
|
||||
{
|
||||
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);
|
||||
CompressedWriteBuffer out2(out1);
|
||||
NativeBlockOutputStream out3(out2, ClickHouseRevision::get());
|
||||
@ -114,7 +114,7 @@ try
|
||||
ReadBufferFromFileDescriptor in1(STDIN_FILENO);
|
||||
CompressedReadBuffer in2(in1);
|
||||
NativeBlockInputStream in3(in2, ClickHouseRevision::get());
|
||||
SharedPtr<IBlockOutputStream> out = table->write({}, {});
|
||||
BlockOutputStreamPtr out = table->write({}, {});
|
||||
copyData(in3, *out);
|
||||
}
|
||||
|
||||
|
@ -143,7 +143,7 @@ try
|
||||
|
||||
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<MergeSortingBlockInputStream>(in, sort_columns, DEFAULT_BLOCK_SIZE, 0, 0, "");
|
||||
//in = std::make_shared<LimitBlockInputStream>(in, 10, 0);
|
||||
|
@ -12,9 +12,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
std::string DataTypeAggregateFunction::getName() const
|
||||
{
|
||||
std::stringstream stream;
|
||||
|
@ -1,5 +1,3 @@
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/IO/WriteBuffer.h>
|
||||
#include <DB/IO/WriteHelpers.h>
|
||||
|
||||
@ -17,8 +15,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_READ_ALL_DATA;
|
||||
|
@ -1,5 +1,3 @@
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Core/Defines.h>
|
||||
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
|
@ -12,60 +12,60 @@
|
||||
|
||||
|
||||
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;
|
||||
size_t n = 50000000;
|
||||
const char * s = "";
|
||||
size_t size = strlen(s) + 1;
|
||||
DB::DataTypeString data_type;
|
||||
std::shared_ptr<ColumnString> column = std::make_shared<ColumnString>();
|
||||
ColumnString::Chars_t & data = column->getChars();
|
||||
ColumnString::Offsets_t & offsets = column->getOffsets();
|
||||
|
||||
data.resize(n * size);
|
||||
offsets.resize(n);
|
||||
for (size_t i = 0; i < n; ++i)
|
||||
{
|
||||
Poco::SharedPtr<DB::ColumnString> column = new DB::ColumnString();
|
||||
DB::ColumnString::Chars_t & data = column->getChars();
|
||||
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;
|
||||
memcpy(&data[i * size], s, size);
|
||||
offsets[i] = (i + 1) * size;
|
||||
}
|
||||
|
||||
{
|
||||
Poco::SharedPtr<DB::ColumnString> column = new DB::ColumnString();
|
||||
std::ofstream ostr("test");
|
||||
WriteBufferFromOStream out_buf(ostr);
|
||||
|
||||
std::ifstream istr("test");
|
||||
DB::ReadBufferFromIStream in_buf(istr);
|
||||
stopwatch.restart();
|
||||
data_type.serializeBinary(*column, out_buf);
|
||||
stopwatch.stop();
|
||||
|
||||
stopwatch.restart();
|
||||
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;
|
||||
}
|
||||
std::cout << "Writing, elapsed: " << stopwatch.elapsedSeconds() << std::endl;
|
||||
}
|
||||
catch (const DB::Exception & e)
|
||||
|
||||
{
|
||||
std::cerr << e.what() << ", " << e.displayText() << std::endl;
|
||||
return 1;
|
||||
std::shared_ptr<ColumnString> column = std::make_shared<ColumnString>();
|
||||
|
||||
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;
|
||||
}
|
||||
catch (const DB::Exception & e)
|
||||
{
|
||||
std::cerr << e.what() << ", " << e.displayText() << std::endl;
|
||||
return 1;
|
||||
}
|
||||
|
@ -10,9 +10,11 @@
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
{
|
||||
Poco::SharedPtr<DB::ColumnUInt64> column = new DB::ColumnUInt64();
|
||||
DB::ColumnUInt64::Container_t & vec = column->getData();
|
||||
DB::DataTypeUInt64 data_type;
|
||||
using namespace DB;
|
||||
|
||||
std::shared_ptr<ColumnUInt64> column = std::make_shared<ColumnUInt64>();
|
||||
ColumnUInt64::Container_t & vec = column->getData();
|
||||
DataTypeUInt64 data_type;
|
||||
|
||||
Stopwatch stopwatch;
|
||||
size_t n = 10000000;
|
||||
@ -22,7 +24,7 @@ int main(int argc, char ** argv)
|
||||
vec[i] = i;
|
||||
|
||||
std::ofstream ostr("test");
|
||||
DB::WriteBufferFromOStream out_buf(ostr);
|
||||
WriteBufferFromOStream out_buf(ostr);
|
||||
|
||||
stopwatch.restart();
|
||||
data_type.serializeBinary(*column, out_buf);
|
||||
|
@ -2,7 +2,6 @@
|
||||
#include <set>
|
||||
#include <chrono>
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
#include <Poco/Mutex.h>
|
||||
#include <Poco/File.h>
|
||||
#include <Poco/UUIDGenerator.h>
|
||||
@ -66,8 +65,6 @@ namespace ErrorCodes
|
||||
|
||||
class TableFunctionFactory;
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Набор известных объектов, которые могут быть использованы в запросе.
|
||||
* Разделяемая часть. Порядок членов (порядок их уничтожения) очень важен.
|
||||
@ -94,8 +91,8 @@ struct ContextShared
|
||||
TableFunctionFactory table_function_factory; /// Табличные функции.
|
||||
AggregateFunctionFactory aggregate_function_factory; /// Агрегатные функции.
|
||||
FormatFactory format_factory; /// Форматы.
|
||||
mutable SharedPtr<Dictionaries> dictionaries; /// Словари Метрики. Инициализируются лениво.
|
||||
mutable SharedPtr<ExternalDictionaries> external_dictionaries;
|
||||
mutable std::shared_ptr<Dictionaries> dictionaries; /// Словари Метрики. Инициализируются лениво.
|
||||
mutable std::shared_ptr<ExternalDictionaries> external_dictionaries;
|
||||
Users users; /// Известные пользователи.
|
||||
Quotas quotas; /// Известные квоты на использование ресурсов.
|
||||
mutable UncompressedCachePtr uncompressed_cache; /// Кэш разжатых блоков.
|
||||
@ -116,7 +113,7 @@ struct ContextShared
|
||||
|
||||
/// Кластеры для distributed таблиц
|
||||
/// Создаются при создании Distributed таблиц, так как нужно дождаться пока будут выставлены Settings
|
||||
mutable Poco::SharedPtr<Clusters> clusters;
|
||||
mutable std::shared_ptr<Clusters> clusters;
|
||||
|
||||
Poco::UUIDGenerator uuid_generator;
|
||||
|
||||
@ -747,7 +744,7 @@ const Dictionaries & Context::getDictionariesImpl(const bool throw_on_error) con
|
||||
auto lock = getLock();
|
||||
|
||||
if (!shared->dictionaries)
|
||||
shared->dictionaries = new Dictionaries{throw_on_error};
|
||||
shared->dictionaries = std::make_shared<Dictionaries>(throw_on_error);
|
||||
|
||||
return *shared->dictionaries;
|
||||
}
|
||||
@ -761,7 +758,7 @@ const ExternalDictionaries & Context::getExternalDictionariesImpl(const bool thr
|
||||
{
|
||||
if (!this->global_context)
|
||||
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;
|
||||
@ -921,7 +918,7 @@ const Cluster & Context::getCluster(const std::string & cluster_name) const
|
||||
{
|
||||
auto lock = getLock();
|
||||
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);
|
||||
@ -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);
|
||||
}
|
||||
|
||||
Poco::SharedPtr<Clusters> Context::getClusters() const
|
||||
std::shared_ptr<Clusters> Context::getClusters() const
|
||||
{
|
||||
{
|
||||
auto lock = getLock();
|
||||
if (!shared->clusters)
|
||||
shared->clusters = new Clusters(settings);
|
||||
shared->clusters = std::make_shared<Clusters>(settings);
|
||||
}
|
||||
|
||||
return shared->clusters;
|
||||
@ -947,7 +944,7 @@ Compiler & Context::getCompiler()
|
||||
auto lock = getLock();
|
||||
|
||||
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;
|
||||
}
|
||||
|
@ -15,7 +15,7 @@ using ColumnsWithSortDescriptions = std::vector<std::pair<const IColumn *, SortC
|
||||
|
||||
static inline bool needCollation(const IColumn * column, const SortColumnDescription & description)
|
||||
{
|
||||
if (description.collator.isNull())
|
||||
if (!description.collator)
|
||||
return false;
|
||||
|
||||
if (column->getName() != "ColumnString")
|
||||
|
@ -777,7 +777,7 @@ bool ParserOrderByElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & ma
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
Poco::SharedPtr<Collator> collator = nullptr;
|
||||
std::shared_ptr<Collator> collator;
|
||||
if (collate.ignore(pos, end))
|
||||
{
|
||||
ws.ignore(pos, end);
|
||||
@ -787,7 +787,7 @@ bool ParserOrderByElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & ma
|
||||
return false;
|
||||
|
||||
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);
|
||||
|
@ -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.
|
||||
* В этом случае, результат сжимается несовместимым алгоритмом для внутреннего использования и этот факт не отражается в HTTP заголовках.
|
||||
*/
|
||||
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
|
||||
used_output.out_maybe_compressed = used_output.out;
|
||||
|
||||
|
@ -20,9 +20,9 @@ public:
|
||||
|
||||
struct Output
|
||||
{
|
||||
SharedPtr<WriteBufferFromHTTPServerResponse> out;
|
||||
std::shared_ptr<WriteBufferFromHTTPServerResponse> 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);
|
||||
|
@ -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()
|
||||
{
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <ext/scope_guard.hpp>
|
||||
|
||||
#include <memory>
|
||||
#include <experimental/optional>
|
||||
|
||||
#include <DB/Common/Macros.h>
|
||||
#include <DB/Common/getFQDNOrHostName.h>
|
||||
@ -390,13 +391,13 @@ int Server::main(const std::vector<std::string> & args)
|
||||
new Poco::Net::TCPServerParams);
|
||||
|
||||
/// 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"))
|
||||
{
|
||||
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.setSendTimeout(settings.send_timeout);
|
||||
interserver_io_http_server = new Poco::Net::HTTPServer(
|
||||
interserver_io_http_server.reset(
|
||||
new HTTPRequestHandlerFactory<InterserverIOHTTPHandler>(*this, "InterserverIOHTTPHandler-factory"),
|
||||
server_pool,
|
||||
interserver_io_http_socket,
|
||||
@ -404,7 +405,7 @@ int Server::main(const std::vector<std::string> & args)
|
||||
}
|
||||
|
||||
/// 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");
|
||||
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")));
|
||||
olap_http_socket.setReceiveTimeout(settings.receive_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"),
|
||||
server_pool,
|
||||
olap_http_socket,
|
||||
|
@ -56,8 +56,8 @@ void TCPHandler::runImpl()
|
||||
socket().setSendTimeout(global_settings.send_timeout);
|
||||
socket().setNoDelay(true);
|
||||
|
||||
in = new ReadBufferFromPocoSocket(socket());
|
||||
out = new WriteBufferFromPocoSocket(socket());
|
||||
in = std::make_shared<ReadBufferFromPocoSocket>(socket());
|
||||
out = std::make_shared<WriteBufferFromPocoSocket>(socket());
|
||||
|
||||
if (in->eof())
|
||||
{
|
||||
@ -128,7 +128,7 @@ void TCPHandler::runImpl()
|
||||
/** Исключение во время выполнения запроса (его надо отдать по сети клиенту).
|
||||
* Клиент сможет его принять, если оно не произошло во время отправки другого пакета и клиент ещё не разорвал соединение.
|
||||
*/
|
||||
SharedPtr<Exception> exception;
|
||||
std::unique_ptr<Exception> exception;
|
||||
|
||||
try
|
||||
{
|
||||
@ -148,8 +148,8 @@ void TCPHandler::runImpl()
|
||||
|
||||
/// Очищаем, так как, получая данные внешних таблиц, мы получили пустой блок.
|
||||
/// А значит, stream помечен как cancelled и читать из него нельзя.
|
||||
state.block_in = nullptr;
|
||||
state.maybe_compressed_in = nullptr; /// Для более корректного учёта MemoryTracker-ом.
|
||||
state.block_in.reset();
|
||||
state.maybe_compressed_in.reset(); /// Для более корректного учёта MemoryTracker-ом.
|
||||
|
||||
/// Обрабатываем Query
|
||||
state.io = executeQuery(state.query, query_context, false, state.stage);
|
||||
@ -173,7 +173,7 @@ void TCPHandler::runImpl()
|
||||
catch (const Exception & e)
|
||||
{
|
||||
state.io.onException();
|
||||
exception = e.clone();
|
||||
exception.reset(e.clone());
|
||||
|
||||
if (e.code() == ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT)
|
||||
throw;
|
||||
@ -187,22 +187,22 @@ void TCPHandler::runImpl()
|
||||
* Будем пытаться отправить эксепшен клиенту в любом случае - см. ниже.
|
||||
*/
|
||||
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)
|
||||
{
|
||||
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)
|
||||
{
|
||||
state.io.onException();
|
||||
exception = new Exception(e.what(), ErrorCodes::STD_EXCEPTION);
|
||||
exception = std::make_unique<Exception>(e.what(), ErrorCodes::STD_EXCEPTION);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
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;
|
||||
@ -609,7 +609,7 @@ void TCPHandler::initBlockInput()
|
||||
if (!state.block_in)
|
||||
{
|
||||
if (state.compression == Protocol::Compression::Enable)
|
||||
state.maybe_compressed_in = new CompressedReadBuffer(*in);
|
||||
state.maybe_compressed_in = std::make_shared<CompressedReadBuffer>(*in);
|
||||
else
|
||||
state.maybe_compressed_in = in;
|
||||
|
||||
@ -625,7 +625,8 @@ void TCPHandler::initBlockOutput()
|
||||
if (!state.block_out)
|
||||
{
|
||||
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
|
||||
state.maybe_compressed_out = out;
|
||||
|
||||
|
@ -28,11 +28,11 @@ struct QueryState
|
||||
Protocol::Compression::Enum compression = Protocol::Compression::Disable;
|
||||
|
||||
/// Откуда читать данные для INSERT-а.
|
||||
SharedPtr<ReadBuffer> maybe_compressed_in;
|
||||
shared_ptr<ReadBuffer> maybe_compressed_in;
|
||||
BlockInputStreamPtr block_in;
|
||||
|
||||
/// Куда писать возвращаемые данные.
|
||||
SharedPtr<WriteBuffer> maybe_compressed_out;
|
||||
shared_ptr<WriteBuffer> maybe_compressed_out;
|
||||
BlockOutputStreamPtr block_out;
|
||||
|
||||
/// Текст запроса.
|
||||
@ -87,8 +87,8 @@ private:
|
||||
Context query_context;
|
||||
|
||||
/// Потоки для чтения/записи из/в сокет соединения с клиентом.
|
||||
SharedPtr<ReadBuffer> in;
|
||||
SharedPtr<WriteBuffer> out;
|
||||
std::shared_ptr<ReadBuffer> in;
|
||||
std::shared_ptr<WriteBuffer> out;
|
||||
|
||||
/// Время после последней проверки остановки запроса и отправки прогресса.
|
||||
Stopwatch after_check_cancelled;
|
||||
|
@ -150,7 +150,7 @@ StoragePtr StorageDistributed::create(
|
||||
NamesAndTypesListPtr columns_,
|
||||
const String & remote_database_,
|
||||
const String & remote_table_,
|
||||
SharedPtr<Cluster> & owned_cluster_,
|
||||
std::shared_ptr<Cluster> & owned_cluster_,
|
||||
Context & context_)
|
||||
{
|
||||
auto res = new StorageDistributed{
|
||||
|
@ -32,8 +32,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
|
@ -10,9 +10,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
class MemoryBlockInputStream : public IProfilingBlockInputStream
|
||||
{
|
||||
public:
|
||||
|
@ -31,8 +31,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
|
||||
|
@ -1,5 +1,3 @@
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Common/Exception.h>
|
||||
#include <DB/Columns/ColumnsNumber.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
@ -10,8 +8,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
class NumbersBlockInputStream : public IProfilingBlockInputStream
|
||||
{
|
||||
public:
|
||||
|
@ -14,119 +14,115 @@
|
||||
#include <DB/Parsers/ParserSelectQuery.h>
|
||||
#include <DB/Parsers/parseQuery.h>
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
try
|
||||
{
|
||||
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>();
|
||||
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>())));
|
||||
block.insert(column1);
|
||||
|
||||
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 = "";
|
||||
String primary_expr_str = "d";
|
||||
const char * begin = primary_expr_str.data();
|
||||
const char * end = begin + primary_expr_str.size();
|
||||
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;
|
||||
ParserExpressionList parser(false);
|
||||
if (!parser.parse(begin, end, primary_expr, max_parsed_pos, expected))
|
||||
ParserSelectQuery parser;
|
||||
if (!parser.parse(begin, end, select, max_parsed_pos, expected))
|
||||
throw Poco::Exception("Cannot parse " + primary_expr_str);
|
||||
|
||||
MergeTreeData::MergingParams params;
|
||||
params.mode = MergeTreeData::MergingParams::Ordinary;
|
||||
BlockInputStreamPtr in = table->read(column_names, select, context, Settings(), stage)[0];
|
||||
|
||||
StoragePtr table = StorageMergeTree::create(
|
||||
"./", "default", "test",
|
||||
names_and_types, {}, {}, ColumnDefaults{},
|
||||
context, primary_expr, "d",
|
||||
nullptr, 101, params, {});
|
||||
|
||||
/// пишем в неё
|
||||
Block sample;
|
||||
{
|
||||
Block block;
|
||||
|
||||
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;
|
||||
|
||||
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);
|
||||
ColumnWithTypeAndName col;
|
||||
col.type = names_and_types->front().type;
|
||||
sample.insert(col);
|
||||
}
|
||||
{
|
||||
ColumnWithTypeAndName col;
|
||||
col.type = names_and_types->back().type;
|
||||
sample.insert(col);
|
||||
}
|
||||
|
||||
/// читаем из неё
|
||||
{
|
||||
Names column_names;
|
||||
column_names.push_back("d");
|
||||
column_names.push_back("a");
|
||||
WriteBufferFromFileDescriptor out_buf(STDOUT_FILENO);
|
||||
|
||||
QueryProcessingStage::Enum stage;
|
||||
RowOutputStreamPtr output_ = std::make_shared<TabSeparatedRowOutputStream>(out_buf, sample);
|
||||
BlockOutputStreamFromRowOutputStream output(output_);
|
||||
|
||||
ASTPtr select;
|
||||
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;
|
||||
copyData(*in, output);
|
||||
}
|
||||
|
||||
return 0;
|
||||
}
|
||||
catch (const DB::Exception & e)
|
||||
{
|
||||
std::cerr << e.displayText() << ", stack trace: \n\n" << e.getStackTrace().toString() << std::endl;
|
||||
throw;
|
||||
}
|
||||
|
@ -54,7 +54,7 @@ try
|
||||
|
||||
block.insert(column2);
|
||||
|
||||
SharedPtr<IBlockOutputStream> out = table->write({}, {});
|
||||
BlockOutputStreamPtr out = table->write({}, {});
|
||||
out->write(block);
|
||||
}
|
||||
|
||||
@ -66,7 +66,7 @@ try
|
||||
|
||||
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;
|
||||
{
|
||||
|
@ -263,7 +263,7 @@ StoragePtr TableFunctionRemote::execute(ASTPtr ast_function, Context & context)
|
||||
if (names.empty())
|
||||
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(
|
||||
getName(),
|
||||
|
@ -63,7 +63,7 @@ StoragePtr TableFunctionShardByHash::execute(ASTPtr ast_function, Context & cont
|
||||
const Cluster & cluster = context.getCluster(cluster_name);
|
||||
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(
|
||||
getName(),
|
||||
|
Loading…
Reference in New Issue
Block a user