mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-20 06:32:08 +00:00
Merge
This commit is contained in:
commit
18256baf7a
@ -8,6 +8,7 @@ Download HP Vertica 7.1.1 Analytic Database Server, Debian or Ubuntu 14.04 versi
|
||||
sudo apt-get install sysstat pstack mcelog
|
||||
sudo dpkg -i vertica_7.1.1-0_amd64.deb
|
||||
sudo sh -c "echo 'export TZ=Europe/Moscow' >> /home/dbadmin/.bash_profile"
|
||||
# Don't specify localhost due to poor support of IPv6.
|
||||
sudo /opt/vertica/sbin/install_vertica --hosts=127.0.0.1 --failure-threshold=NONE
|
||||
sudo mkdir /opt/vertica-data/
|
||||
sudo chown dbadmin /opt/vertica-data/
|
||||
@ -23,8 +24,6 @@ both directories: /opt/vertica-data/
|
||||
main menu
|
||||
exit
|
||||
|
||||
PS. Note that Vertica doesn't support IPv6.
|
||||
|
||||
How to prepare data
|
||||
-------------------
|
||||
|
||||
|
@ -26,13 +26,13 @@ class IConnectionPool : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
typedef PoolBase<Connection>::Entry Entry;
|
||||
virtual Entry get(Settings * settings = nullptr) = 0;
|
||||
virtual Entry get(const Settings * settings = nullptr) = 0;
|
||||
|
||||
/** Выделяет до указанного количества соединений для работы.
|
||||
* Соединения предоставляют доступ к разным репликам одного шарда.
|
||||
* Выкидывает исключение, если не удалось выделить ни одного соединения.
|
||||
*/
|
||||
virtual std::vector<Entry> getMany(Settings * settings = nullptr)
|
||||
virtual std::vector<Entry> getMany(const Settings * settings = nullptr)
|
||||
{
|
||||
return std::vector<Entry>{ get(settings) };
|
||||
}
|
||||
@ -89,7 +89,7 @@ public:
|
||||
|
||||
|
||||
/** Выделяет соединение для работы. */
|
||||
Entry get(Settings * settings = nullptr) override
|
||||
Entry get(const Settings * settings = nullptr) override
|
||||
{
|
||||
if (settings)
|
||||
return Base::get(settings->queue_max_wait_ms.totalMilliseconds());
|
||||
|
@ -21,11 +21,11 @@ namespace DB
|
||||
*
|
||||
* Замечание: если один из вложенных пулов заблокируется из-за переполнения, то этот пул тоже заблокируется.
|
||||
*/
|
||||
class ConnectionPoolWithFailover : public PoolWithFailoverBase<IConnectionPool, Settings *>, public IConnectionPool
|
||||
class ConnectionPoolWithFailover : public PoolWithFailoverBase<IConnectionPool>, public IConnectionPool
|
||||
{
|
||||
public:
|
||||
typedef IConnectionPool::Entry Entry;
|
||||
typedef PoolWithFailoverBase<IConnectionPool, Settings *> Base;
|
||||
typedef PoolWithFailoverBase<IConnectionPool> Base;
|
||||
|
||||
ConnectionPoolWithFailover(ConnectionPools & nested_pools_,
|
||||
LoadBalancing load_balancing,
|
||||
@ -52,7 +52,7 @@ public:
|
||||
}
|
||||
|
||||
/** Выделяет соединение для работы. */
|
||||
Entry get(Settings * settings = nullptr) override
|
||||
Entry get(const Settings * settings = nullptr) override
|
||||
{
|
||||
applyLoadBalancing(settings);
|
||||
return Base::get(settings);
|
||||
@ -61,14 +61,14 @@ public:
|
||||
/** Выделяет до указанного количества соединений для работы.
|
||||
* Соединения предоставляют доступ к разным репликам одного шарда.
|
||||
*/
|
||||
std::vector<Entry> getMany(Settings * settings = nullptr) override
|
||||
std::vector<Entry> getMany(const Settings * settings = nullptr) override
|
||||
{
|
||||
applyLoadBalancing(settings);
|
||||
return Base::getMany(settings);
|
||||
}
|
||||
|
||||
protected:
|
||||
bool tryGet(ConnectionPoolPtr pool, Settings * settings, Entry & out_entry, std::stringstream & fail_message) override
|
||||
bool tryGet(ConnectionPoolPtr pool, const Settings * settings, Entry & out_entry, std::stringstream & fail_message) override
|
||||
{
|
||||
try
|
||||
{
|
||||
@ -90,7 +90,7 @@ private:
|
||||
std::vector<size_t> hostname_differences; /// Расстояния от имени этого хоста до имен хостов пулов.
|
||||
LoadBalancing default_load_balancing;
|
||||
|
||||
void applyLoadBalancing(Settings * settings)
|
||||
void applyLoadBalancing(const Settings * settings)
|
||||
{
|
||||
LoadBalancing load_balancing = default_load_balancing;
|
||||
if (settings)
|
||||
|
@ -21,10 +21,10 @@ class ParallelReplicas final : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
/// Принимает готовое соединение.
|
||||
ParallelReplicas(Connection * connection_, Settings * settings_, ThrottlerPtr throttler_);
|
||||
ParallelReplicas(Connection * connection_, const Settings * settings_, ThrottlerPtr throttler_);
|
||||
|
||||
/// Принимает пул, из которого нужно будет достать одно или несколько соединений.
|
||||
ParallelReplicas(IConnectionPool * pool_, Settings * settings_, ThrottlerPtr throttler_);
|
||||
ParallelReplicas(IConnectionPool * pool_, const Settings * settings_, ThrottlerPtr throttler_);
|
||||
|
||||
/// Отправить на реплики всё содержимое внешних таблиц.
|
||||
void sendExternalTablesData(std::vector<ExternalTablesData> & data);
|
||||
@ -82,7 +82,7 @@ private:
|
||||
void invalidateReplica(ReplicaMap::iterator it);
|
||||
|
||||
private:
|
||||
Settings * settings;
|
||||
const Settings * settings;
|
||||
ReplicaMap replica_map;
|
||||
|
||||
/// Если не nullptr, то используется, чтобы ограничить сетевой трафик.
|
||||
|
@ -24,36 +24,51 @@ public:
|
||||
};
|
||||
|
||||
|
||||
/** шаблон для столбцов-констант (столбцов одинаковых значений).
|
||||
/** Столбец-константа может содержать внутри себя само значение,
|
||||
* или, в случае массивов, SharedPtr от значения-массива,
|
||||
* чтобы избежать проблем производительности при копировании очень больших массивов.
|
||||
*
|
||||
* T - тип значения,
|
||||
* DataHolder - как значение хранится в таблице (либо T, либо SharedPtr<T>)
|
||||
* Derived должен реализовать методы getDataFromHolderImpl - получить ссылку на значение из holder-а.
|
||||
*
|
||||
* Для строк и массивов реализации sizeOfField и byteSize могут быть некорректными.
|
||||
*/
|
||||
template <typename T>
|
||||
class ColumnConst final : public IColumnConst
|
||||
template <typename T, typename DataHolder, typename Derived>
|
||||
class ColumnConstBase : public IColumnConst
|
||||
{
|
||||
protected:
|
||||
size_t s;
|
||||
DataHolder data;
|
||||
DataTypePtr data_type;
|
||||
|
||||
T & getDataFromHolder() { return static_cast<Derived *>(this)->getDataFromHolderImpl(); }
|
||||
const T & getDataFromHolder() const { return static_cast<const Derived *>(this)->getDataFromHolderImpl(); }
|
||||
|
||||
ColumnConstBase(size_t s_, const DataHolder & data_, DataTypePtr data_type_)
|
||||
: s(s_), data(data_), data_type(data_type_) {}
|
||||
|
||||
public:
|
||||
typedef T Type;
|
||||
typedef typename NearestFieldType<T>::Type FieldType;
|
||||
|
||||
/// Для ColumnConst<Array> data_type_ должен быть ненулевым.
|
||||
/// Для ColumnConst<String> data_type_ должен быть ненулевым, если тип данных FixedString.
|
||||
ColumnConst(size_t s_, const T & data_, DataTypePtr data_type_ = DataTypePtr()) : s(s_), data(data_), data_type(data_type_) {}
|
||||
|
||||
std::string getName() const override { return "ColumnConst<" + TypeName<T>::get() + ">"; }
|
||||
bool isNumeric() const override { return IsNumber<T>::value; }
|
||||
bool isFixed() const override { return IsNumber<T>::value; }
|
||||
size_t sizeOfField() const override { return sizeof(T); }
|
||||
ColumnPtr cloneResized(size_t s_) const override { return new ColumnConst(s_, data, data_type); }
|
||||
ColumnPtr cloneResized(size_t s_) const override { return new Derived(s_, data, data_type); }
|
||||
size_t size() const override { return s; }
|
||||
Field operator[](size_t n) const override { return FieldType(data); }
|
||||
void get(size_t n, Field & res) const override { res = FieldType(data); }
|
||||
Field operator[](size_t n) const override { return FieldType(getDataFromHolder()); }
|
||||
void get(size_t n, Field & res) const override { res = FieldType(getDataFromHolder()); }
|
||||
|
||||
ColumnPtr cut(size_t start, size_t length) const override
|
||||
{
|
||||
return new ColumnConst(length, data, data_type);
|
||||
return new Derived(length, data, data_type);
|
||||
}
|
||||
|
||||
void insert(const Field & x) override
|
||||
{
|
||||
if (x.get<FieldType>() != FieldType(data))
|
||||
if (x.get<FieldType>() != FieldType(getDataFromHolder()))
|
||||
throw Exception("Cannot insert different element into constant column " + getName(),
|
||||
ErrorCodes::CANNOT_INSERT_ELEMENT_INTO_CONSTANT_COLUMN);
|
||||
++s;
|
||||
@ -66,7 +81,7 @@ public:
|
||||
|
||||
void insertFrom(const IColumn & src, size_t n) override
|
||||
{
|
||||
if (data != static_cast<const ColumnConst<T> &>(src).data)
|
||||
if (getDataFromHolder() != static_cast<const Derived &>(src).getDataFromHolder())
|
||||
throw Exception("Cannot insert different element into constant column " + getName(),
|
||||
ErrorCodes::CANNOT_INSERT_ELEMENT_INTO_CONSTANT_COLUMN);
|
||||
++s;
|
||||
@ -79,7 +94,7 @@ public:
|
||||
if (s != filt.size())
|
||||
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
||||
return new ColumnConst(countBytesInFilter(filt), data, data_type);
|
||||
return new Derived(countBytesInFilter(filt), data, data_type);
|
||||
}
|
||||
|
||||
ColumnPtr replicate(const Offsets_t & offsets) const override
|
||||
@ -88,7 +103,7 @@ public:
|
||||
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
||||
size_t replicated_size = 0 == s ? 0 : offsets.back();
|
||||
return new ColumnConst(replicated_size, data, data_type);
|
||||
return new Derived(replicated_size, data, data_type);
|
||||
}
|
||||
|
||||
size_t byteSize() const override { return sizeof(data) + sizeof(s); }
|
||||
@ -103,13 +118,13 @@ public:
|
||||
if (perm.size() < limit)
|
||||
throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
||||
return new ColumnConst(limit, data, data_type);
|
||||
return new Derived(limit, data, data_type);
|
||||
}
|
||||
|
||||
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override
|
||||
{
|
||||
const ColumnConst<T> & rhs = static_cast<const ColumnConst<T> &>(rhs_);
|
||||
return data < rhs.data /// TODO: правильное сравнение NaN-ов в константных столбцах.
|
||||
const Derived & rhs = static_cast<const Derived &>(rhs_);
|
||||
return getDataFromHolder() < rhs.getDataFromHolder() /// TODO: правильное сравнение NaN-ов в константных столбцах.
|
||||
? -1
|
||||
: (data == rhs.data
|
||||
? 0
|
||||
@ -123,30 +138,79 @@ public:
|
||||
res[i] = i;
|
||||
}
|
||||
|
||||
DataTypePtr & getDataType() { return data_type; }
|
||||
const DataTypePtr & getDataType() const { return data_type; }
|
||||
};
|
||||
|
||||
|
||||
/** шаблон для столбцов-констант (столбцов одинаковых значений).
|
||||
*/
|
||||
template <typename T>
|
||||
class ColumnConst final : public ColumnConstBase<T, T, ColumnConst<T>>
|
||||
{
|
||||
private:
|
||||
friend class ColumnConstBase<T, T, ColumnConst<T>>;
|
||||
|
||||
T & getDataFromHolderImpl() { return this->data; }
|
||||
const T & getDataFromHolderImpl() const { return this->data; }
|
||||
|
||||
public:
|
||||
/// Для ColumnConst<Array> data_type_ должен быть ненулевым.
|
||||
/// Для ColumnConst<String> data_type_ должен быть ненулевым, если тип данных FixedString.
|
||||
ColumnConst(size_t s_, const T & data_, DataTypePtr data_type_ = DataTypePtr())
|
||||
: ColumnConstBase<T, T, ColumnConst<T>>(s_, data_, data_type_) {}
|
||||
|
||||
StringRef getDataAt(size_t n) const override;
|
||||
StringRef getDataAtWithTerminatingZero(size_t n) const override;
|
||||
UInt64 get64(size_t n) const override;
|
||||
|
||||
/** Более эффективные методы манипуляции */
|
||||
T & getData() { return data; }
|
||||
const T & getData() const { return data; }
|
||||
T & getData() { return this->data; }
|
||||
const T & getData() const { return this->data; }
|
||||
|
||||
/** Преобразование из константы в полноценный столбец */
|
||||
ColumnPtr convertToFullColumn() const override;
|
||||
|
||||
void getExtremes(Field & min, Field & max) const override
|
||||
{
|
||||
min = FieldType(data);
|
||||
max = FieldType(data);
|
||||
min = typename ColumnConstBase<T, T, ColumnConst<T>>::FieldType(this->data);
|
||||
max = typename ColumnConstBase<T, T, ColumnConst<T>>::FieldType(this->data);
|
||||
}
|
||||
};
|
||||
|
||||
DataTypePtr & getDataType() { return data_type; }
|
||||
const DataTypePtr & getDataType() const { return data_type; }
|
||||
|
||||
template <>
|
||||
class ColumnConst<Array> final : public ColumnConstBase<Array, SharedPtr<Array>, ColumnConst<Array>>
|
||||
{
|
||||
private:
|
||||
size_t s;
|
||||
T data;
|
||||
DataTypePtr data_type;
|
||||
friend class ColumnConstBase<Array, SharedPtr<Array>, ColumnConst<Array>>;
|
||||
|
||||
Array & getDataFromHolderImpl() { return *data; }
|
||||
const Array & getDataFromHolderImpl() const { return *data; }
|
||||
|
||||
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_) {}
|
||||
|
||||
ColumnConst(size_t s_, const SharedPtr<Array> & data_, DataTypePtr data_type_ = DataTypePtr())
|
||||
: ColumnConstBase<Array, SharedPtr<Array>, ColumnConst<Array>>(s_, data_, data_type_) {}
|
||||
|
||||
StringRef getDataAt(size_t n) const override;
|
||||
StringRef getDataAtWithTerminatingZero(size_t n) const override;
|
||||
UInt64 get64(size_t n) const override;
|
||||
|
||||
/** Более эффективные методы манипуляции */
|
||||
const Array & getData() const { return *data; }
|
||||
|
||||
/** Преобразование из константы в полноценный столбец */
|
||||
ColumnPtr convertToFullColumn() const override;
|
||||
|
||||
void getExtremes(Field & min, Field & max) const override
|
||||
{
|
||||
min = FieldType();
|
||||
max = FieldType();
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
@ -158,19 +222,17 @@ template <typename T> ColumnPtr ColumnConst<T>::convertToFullColumn() const
|
||||
{
|
||||
ColumnVector<T> * res_ = new ColumnVector<T>;
|
||||
ColumnPtr res = res_;
|
||||
res_->getData().assign(s, data);
|
||||
res_->getData().assign(this->s, this->data);
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
template <> ColumnPtr ColumnConst<String>::convertToFullColumn() const;
|
||||
|
||||
template <> ColumnPtr ColumnConst<Array>::convertToFullColumn() const;
|
||||
|
||||
|
||||
template <typename T> StringRef ColumnConst<T>::getDataAt(size_t n) const
|
||||
{
|
||||
throw Exception("Method getDataAt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
throw Exception("Method getDataAt is not supported for " + this->getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
template <> inline StringRef ColumnConst<String>::getDataAt(size_t n) const
|
||||
@ -180,7 +242,7 @@ template <> inline StringRef ColumnConst<String>::getDataAt(size_t n) const
|
||||
|
||||
template <typename T> UInt64 ColumnConst<T>::get64(size_t n) const
|
||||
{
|
||||
throw Exception("Method get64 is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
throw Exception("Method get64 is not supported for " + this->getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
/// Для элементарных типов.
|
||||
|
@ -1,25 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Columns/IColumnDummy.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Содержит промежуточные данные для вычисления выражений в функциях высшего порядка.
|
||||
* Это - вложенный столбец произвольного размера.
|
||||
* Сам ColumnReplicated притворяется, как столбец указанного в конструкторе размера.
|
||||
*/
|
||||
class ColumnReplicated final : public IColumnDummy
|
||||
{
|
||||
public:
|
||||
ColumnReplicated(size_t s_, ColumnPtr nested_) : IColumnDummy(s_), nested(nested_) {}
|
||||
std::string getName() const override { return "ColumnReplicated"; }
|
||||
ColumnPtr cloneDummy(size_t s_) const override { return new ColumnReplicated(s_, nested); }
|
||||
|
||||
ColumnPtr & getData() { return nested; }
|
||||
private:
|
||||
ColumnPtr nested;
|
||||
};
|
||||
|
||||
}
|
@ -41,12 +41,7 @@ public:
|
||||
|
||||
ColumnPtr filter(const Filter & filt) const override
|
||||
{
|
||||
size_t new_size = 0;
|
||||
for (Filter::const_iterator it = filt.begin(); it != filt.end(); ++it)
|
||||
if (*it)
|
||||
++new_size;
|
||||
|
||||
return cloneDummy(new_size);
|
||||
return cloneDummy(countBytesInFilter(filt));
|
||||
}
|
||||
|
||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override
|
||||
|
@ -38,6 +38,11 @@ public:
|
||||
{
|
||||
return amount;
|
||||
}
|
||||
|
||||
Int64 getPeak() const
|
||||
{
|
||||
return peak;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
|
@ -13,6 +13,7 @@
|
||||
|
||||
#include <stdint.h>
|
||||
#include <cstdlib>
|
||||
#include <stddef.h>
|
||||
|
||||
#define ROTL(x,b) (u64)( ((x) << (b)) | ( (x) >> (64 - (b))) )
|
||||
|
||||
|
@ -78,5 +78,7 @@
|
||||
/// Граница, на которых должны быть выровнены блоки для асинхронных файловых операций.
|
||||
#define DEFAULT_AIO_FILE_BLOCK_SIZE 4096
|
||||
|
||||
#define DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS_STR "7500"
|
||||
|
||||
#define ALWAYS_INLINE __attribute__((__always_inline__))
|
||||
#define NO_INLINE __attribute__((__noinline__))
|
||||
|
@ -31,6 +31,8 @@ ExceptionPtr cloneCurrentException();
|
||||
void tryLogCurrentException(const char * log_name);
|
||||
void tryLogCurrentException(Poco::Logger * logger);
|
||||
|
||||
std::string getCurrentExceptionMessage(bool with_stacktrace);
|
||||
|
||||
|
||||
void rethrowFirstException(Exceptions & exceptions);
|
||||
|
||||
|
@ -2,12 +2,13 @@
|
||||
|
||||
#include <DB/DataStreams/IBlockInputStream.h>
|
||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||
#include <DB/Interpreters/ProcessList.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ProcessListEntry;
|
||||
|
||||
struct BlockIO
|
||||
{
|
||||
/** process_list_entry должен уничтожаться позже, чем in и out,
|
||||
@ -15,7 +16,7 @@ struct BlockIO
|
||||
* (MemoryTracker * current_memory_tracker),
|
||||
* которая может использоваться до уничтожения in и out.
|
||||
*/
|
||||
ProcessList::EntryPtr process_list_entry;
|
||||
std::shared_ptr<ProcessListEntry> process_list_entry;
|
||||
|
||||
BlockInputStreamPtr in;
|
||||
BlockOutputStreamPtr out;
|
||||
@ -23,6 +24,25 @@ struct BlockIO
|
||||
Block in_sample; /// Пример блока, который будет прочитан из in.
|
||||
Block out_sample; /// Пример блока, которого нужно писать в out.
|
||||
|
||||
/// Здесь могут быть установлены колбэки для логгирования запроса.
|
||||
std::function<void(IBlockInputStream &)> finish_callback;
|
||||
std::function<void()> exception_callback;
|
||||
|
||||
/// Вызывайте эти функции, если нужно логгировать запрос.
|
||||
void onFinish()
|
||||
{
|
||||
if (in && finish_callback)
|
||||
finish_callback(*in);
|
||||
}
|
||||
|
||||
void onException()
|
||||
{
|
||||
if (exception_callback)
|
||||
exception_callback();
|
||||
else
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
|
||||
BlockIO & operator= (const BlockIO & rhs)
|
||||
{
|
||||
/// Обеспечиваем правильный порядок уничтожения.
|
||||
@ -36,8 +56,13 @@ struct BlockIO
|
||||
in_sample = rhs.in_sample;
|
||||
out_sample = rhs.out_sample;
|
||||
|
||||
finish_callback = rhs.finish_callback;
|
||||
exception_callback = rhs.exception_callback;
|
||||
|
||||
return *this;
|
||||
}
|
||||
|
||||
~BlockIO();
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <DB/Core/Progress.h>
|
||||
|
||||
#include <DB/Interpreters/Limits.h>
|
||||
#include <DB/Interpreters/ProcessList.h>
|
||||
|
||||
#include <DB/DataStreams/BlockStreamProfileInfo.h>
|
||||
#include <DB/DataStreams/IBlockInputStream.h>
|
||||
@ -14,6 +13,7 @@ namespace DB
|
||||
{
|
||||
|
||||
class QuotaForIntervals;
|
||||
class ProcessListElement;
|
||||
|
||||
|
||||
/** Смотрит за тем, как работает источник блоков.
|
||||
@ -82,7 +82,7 @@ public:
|
||||
* На основе этой информации будет проверяться квота, и некоторые ограничения.
|
||||
* Также эта информация будет доступна в запросе SHOW PROCESSLIST.
|
||||
*/
|
||||
void setProcessListElement(ProcessList::Element * elem);
|
||||
void setProcessListElement(ProcessListElement * elem);
|
||||
|
||||
/** Установить информацию о приблизительном общем количестве строк, которых нужно прочитать.
|
||||
*/
|
||||
@ -154,7 +154,7 @@ protected:
|
||||
BlockStreamProfileInfo info;
|
||||
std::atomic<bool> is_cancelled{false};
|
||||
ProgressCallback progress_callback;
|
||||
ProcessList::Element * process_list_elem = nullptr;
|
||||
ProcessListElement * process_list_elem = nullptr;
|
||||
|
||||
bool enabled_extremes = false;
|
||||
|
||||
|
@ -41,7 +41,7 @@ public:
|
||||
{
|
||||
BlockInputStreamPtr from = new OneBlockInputStream(block);
|
||||
InterpreterSelectQuery select(queries[i], context, QueryProcessingStage::Complete, 0, from);
|
||||
BlockInputStreamPtr data = new MaterializingBlockInputStream(select.execute());
|
||||
BlockInputStreamPtr data = new MaterializingBlockInputStream(select.execute().in);
|
||||
copyData(*data, *children[i]);
|
||||
}
|
||||
|
||||
|
@ -136,6 +136,9 @@ protected:
|
||||
{
|
||||
createParallelReplicas();
|
||||
|
||||
if (settings.skip_unavailable_shards && 0 == parallel_replicas->size())
|
||||
return Block();
|
||||
|
||||
established = true;
|
||||
|
||||
parallel_replicas->sendQuery(query, "", stage, true);
|
||||
|
@ -15,7 +15,7 @@ namespace DB
|
||||
class RemoteBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
RemoteBlockOutputStream(Connection & connection_, const String & query_, Settings * settings_ = nullptr)
|
||||
RemoteBlockOutputStream(Connection & connection_, const String & query_, const Settings * settings_ = nullptr)
|
||||
: connection(connection_), query(query_), settings(settings_)
|
||||
{
|
||||
}
|
||||
@ -105,7 +105,7 @@ public:
|
||||
private:
|
||||
Connection & connection;
|
||||
String query;
|
||||
Settings * settings;
|
||||
const Settings * settings;
|
||||
Block sample_block;
|
||||
};
|
||||
|
||||
|
@ -4,20 +4,25 @@
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/DataTypes/DataTypeArray.h>
|
||||
|
||||
#include <DB/Core/Exception.h>
|
||||
#include <DB/Core/ErrorCodes.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Для заданного значения Field возвращает минимальный тип данных, позволяющий хранить значение этого типа.
|
||||
/** Для заданного значения Field возвращает минимальный тип данных, позволяющий хранить значение этого типа.
|
||||
* В случае, если Field - массив, конвертирует все элементы к общему типу.
|
||||
*/
|
||||
class FieldToDataType : public StaticVisitor<DataTypePtr>
|
||||
{
|
||||
public:
|
||||
DataTypePtr operator() (const Null & x) const
|
||||
DataTypePtr operator() (Null & x) const
|
||||
{
|
||||
throw Exception("NULL literals is not implemented yet", ErrorCodes::NOT_IMPLEMENTED);
|
||||
throw Exception("NULL literals are not implemented yet", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
DataTypePtr operator() (const UInt64 & x) const
|
||||
DataTypePtr operator() (UInt64 & x) const
|
||||
{
|
||||
if (x <= std::numeric_limits<UInt8>::max()) return new DataTypeUInt8;
|
||||
if (x <= std::numeric_limits<UInt16>::max()) return new DataTypeUInt16;
|
||||
@ -25,7 +30,7 @@ public:
|
||||
return new DataTypeUInt64;
|
||||
}
|
||||
|
||||
DataTypePtr operator() (const Int64 & x) const
|
||||
DataTypePtr operator() (Int64 & x) const
|
||||
{
|
||||
if (x <= std::numeric_limits<Int8>::max() && x >= std::numeric_limits<Int8>::min()) return new DataTypeInt8;
|
||||
if (x <= std::numeric_limits<Int16>::max() && x >= std::numeric_limits<Int16>::min()) return new DataTypeInt16;
|
||||
@ -33,22 +38,18 @@ public:
|
||||
return new DataTypeInt64;
|
||||
}
|
||||
|
||||
DataTypePtr operator() (const Float64 & x) const
|
||||
DataTypePtr operator() (Float64 & x) const
|
||||
{
|
||||
return new DataTypeFloat64;
|
||||
}
|
||||
|
||||
DataTypePtr operator() (const String & x) const
|
||||
DataTypePtr operator() (String & x) const
|
||||
{
|
||||
return new DataTypeString;
|
||||
}
|
||||
|
||||
DataTypePtr operator() (const Array & x) const
|
||||
{
|
||||
return new DataTypeArray(apply_visitor(FieldToDataType(), x.at(0)));
|
||||
}
|
||||
DataTypePtr operator() (Array & x) const;
|
||||
};
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypeArray.h>
|
||||
#include <DB/Columns/ColumnVector.h>
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
@ -14,7 +15,7 @@ namespace DB
|
||||
|
||||
/** Функция выбора по условию: if(cond, then, else).
|
||||
* cond - UInt8
|
||||
* then, else - либо числа/даты/даты-с-временем, либо строки.
|
||||
* then, else - числовые типы, для которых есть общий тип, либо даты, даты-с-временем, либо строки, либо массивы таких типов.
|
||||
*/
|
||||
|
||||
|
||||
@ -275,6 +276,425 @@ struct StringIfImpl
|
||||
};
|
||||
|
||||
|
||||
template <typename A, typename B, typename ResultType>
|
||||
struct NumArrayIfImpl
|
||||
{
|
||||
template <typename FromT>
|
||||
static ALWAYS_INLINE void copy_from_vector(
|
||||
size_t i,
|
||||
const PODArray<FromT> & from_data, const ColumnArray::Offsets_t & from_offsets, ColumnArray::Offset_t from_prev_offset,
|
||||
PODArray<ResultType> & to_data, ColumnArray::Offsets_t & to_offsets, ColumnArray::Offset_t & to_prev_offset)
|
||||
{
|
||||
size_t size_to_write = from_offsets[i] - from_prev_offset;
|
||||
to_data.resize(to_data.size() + size_to_write);
|
||||
|
||||
for (size_t i = 0; i < size_to_write; ++i)
|
||||
to_data[to_prev_offset + i] = static_cast<ResultType>(from_data[from_prev_offset + i]);
|
||||
|
||||
to_prev_offset += size_to_write;
|
||||
to_offsets[i] = to_prev_offset;
|
||||
}
|
||||
|
||||
static ALWAYS_INLINE void copy_from_constant(
|
||||
size_t i,
|
||||
const PODArray<ResultType> & from_data,
|
||||
PODArray<ResultType> & to_data, ColumnArray::Offsets_t & to_offsets, ColumnArray::Offset_t & to_prev_offset)
|
||||
{
|
||||
size_t size_to_write = from_data.size();
|
||||
to_data.resize(to_data.size() + size_to_write);
|
||||
memcpy(&to_data[to_prev_offset], from_data.data(), size_to_write * sizeof(from_data[0]));
|
||||
to_prev_offset += size_to_write;
|
||||
to_offsets[i] = to_prev_offset;
|
||||
}
|
||||
|
||||
static void create_result_column(
|
||||
Block & block, size_t result,
|
||||
PODArray<ResultType> ** c_data, ColumnArray::Offsets_t ** c_offsets)
|
||||
{
|
||||
ColumnVector<ResultType> * col_res_vec = new ColumnVector<ResultType>;
|
||||
ColumnArray * col_res_array = new ColumnArray(col_res_vec);
|
||||
block.getByPosition(result).column = col_res_array;
|
||||
|
||||
*c_data = &col_res_vec->getData();
|
||||
*c_offsets = &col_res_array->getOffsets();
|
||||
}
|
||||
|
||||
|
||||
static void vector_vector(
|
||||
const PODArray<UInt8> & cond,
|
||||
const PODArray<A> & a_data, const ColumnArray::Offsets_t & a_offsets,
|
||||
const PODArray<B> & b_data, const ColumnArray::Offsets_t & b_offsets,
|
||||
Block & block, size_t result)
|
||||
{
|
||||
PODArray<ResultType> * c_data = nullptr;
|
||||
ColumnArray::Offsets_t * c_offsets = nullptr;
|
||||
create_result_column(block, result, &c_data, &c_offsets);
|
||||
|
||||
size_t size = cond.size();
|
||||
c_offsets->resize(size);
|
||||
c_data->reserve(std::max(a_data.size(), b_data.size()));
|
||||
|
||||
ColumnArray::Offset_t a_prev_offset = 0;
|
||||
ColumnArray::Offset_t b_prev_offset = 0;
|
||||
ColumnArray::Offset_t c_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (cond[i])
|
||||
copy_from_vector(i, a_data, a_offsets, a_prev_offset, *c_data, *c_offsets, c_prev_offset);
|
||||
else
|
||||
copy_from_vector(i, b_data, b_offsets, b_prev_offset, *c_data, *c_offsets, c_prev_offset);
|
||||
|
||||
a_prev_offset = a_offsets[i];
|
||||
b_prev_offset = b_offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
static void vector_constant(
|
||||
const PODArray<UInt8> & cond,
|
||||
const PODArray<A> & a_data, const ColumnArray::Offsets_t & a_offsets,
|
||||
const Array & b,
|
||||
Block & block, size_t result)
|
||||
{
|
||||
PODArray<ResultType> * c_data = nullptr;
|
||||
ColumnArray::Offsets_t * c_offsets = nullptr;
|
||||
create_result_column(block, result, &c_data, &c_offsets);
|
||||
|
||||
PODArray<ResultType> b_converted(b.size());
|
||||
for (size_t i = 0, size = b.size(); i < size; ++i)
|
||||
b_converted[i] = b[i].get<typename NearestFieldType<B>::Type>();
|
||||
|
||||
size_t size = cond.size();
|
||||
c_offsets->resize(size);
|
||||
c_data->reserve(a_data.size());
|
||||
|
||||
ColumnArray::Offset_t a_prev_offset = 0;
|
||||
ColumnArray::Offset_t c_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (cond[i])
|
||||
copy_from_vector(i, a_data, a_offsets, a_prev_offset, *c_data, *c_offsets, c_prev_offset);
|
||||
else
|
||||
copy_from_constant(i, b_converted, *c_data, *c_offsets, c_prev_offset);
|
||||
|
||||
a_prev_offset = a_offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
static void constant_vector(
|
||||
const PODArray<UInt8> & cond,
|
||||
const Array & a,
|
||||
const PODArray<B> & b_data, const ColumnArray::Offsets_t & b_offsets,
|
||||
Block & block, size_t result)
|
||||
{
|
||||
PODArray<ResultType> * c_data = nullptr;
|
||||
ColumnArray::Offsets_t * c_offsets = nullptr;
|
||||
create_result_column(block, result, &c_data, &c_offsets);
|
||||
|
||||
PODArray<ResultType> a_converted(a.size());
|
||||
for (size_t i = 0, size = a.size(); i < size; ++i)
|
||||
a_converted[i] = a[i].get<typename NearestFieldType<A>::Type>();
|
||||
|
||||
size_t size = cond.size();
|
||||
c_offsets->resize(size);
|
||||
c_data->reserve(b_data.size());
|
||||
|
||||
ColumnArray::Offset_t b_prev_offset = 0;
|
||||
ColumnArray::Offset_t c_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (cond[i])
|
||||
copy_from_constant(i, a_converted, *c_data, *c_offsets, c_prev_offset);
|
||||
else
|
||||
copy_from_vector(i, b_data, b_offsets, b_prev_offset, *c_data, *c_offsets, c_prev_offset);
|
||||
|
||||
b_prev_offset = b_offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
static void constant_constant(
|
||||
const PODArray<UInt8> & cond,
|
||||
const Array & a, const Array & b,
|
||||
Block & block, size_t result)
|
||||
{
|
||||
PODArray<ResultType> * c_data = nullptr;
|
||||
ColumnArray::Offsets_t * c_offsets = nullptr;
|
||||
create_result_column(block, result, &c_data, &c_offsets);
|
||||
|
||||
PODArray<ResultType> a_converted(a.size());
|
||||
for (size_t i = 0, size = a.size(); i < size; ++i)
|
||||
a_converted[i] = a[i].get<typename NearestFieldType<A>::Type>();
|
||||
|
||||
PODArray<ResultType> b_converted(b.size());
|
||||
for (size_t i = 0, size = b.size(); i < size; ++i)
|
||||
b_converted[i] = b[i].get<typename NearestFieldType<B>::Type>();
|
||||
|
||||
size_t size = cond.size();
|
||||
c_offsets->resize(size);
|
||||
c_data->reserve((std::max(a.size(), b.size())) * size);
|
||||
|
||||
ColumnArray::Offset_t c_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (cond[i])
|
||||
copy_from_constant(i, a_converted, *c_data, *c_offsets, c_prev_offset);
|
||||
else
|
||||
copy_from_constant(i, b_converted, *c_data, *c_offsets, c_prev_offset);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
template <typename A, typename B>
|
||||
struct NumArrayIfImpl<A, B, NumberTraits::Error>
|
||||
{
|
||||
private:
|
||||
static void throw_error()
|
||||
{
|
||||
throw Exception("Internal logic error: invalid types of arguments 2 and 3 of if", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
public:
|
||||
static void vector_vector(
|
||||
const PODArray<UInt8> & cond,
|
||||
const PODArray<A> & a_data, const ColumnArray::Offsets_t & a_offsets,
|
||||
const PODArray<B> & b_data, const ColumnArray::Offsets_t & b_offsets,
|
||||
Block & block, size_t result)
|
||||
{
|
||||
throw_error();
|
||||
}
|
||||
|
||||
static void vector_constant(
|
||||
const PODArray<UInt8> & cond,
|
||||
const PODArray<A> & a_data, const ColumnArray::Offsets_t & a_offsets,
|
||||
const Array & b,
|
||||
Block & block, size_t result)
|
||||
{
|
||||
throw_error();
|
||||
}
|
||||
|
||||
static void constant_vector(
|
||||
const PODArray<UInt8> & cond,
|
||||
const Array & a,
|
||||
const PODArray<B> & b_data, const ColumnArray::Offsets_t & b_offsets,
|
||||
Block & block, size_t result)
|
||||
{
|
||||
throw_error();
|
||||
}
|
||||
|
||||
static void constant_constant(
|
||||
const PODArray<UInt8> & cond,
|
||||
const Array & a, const Array & b,
|
||||
Block & block, size_t result)
|
||||
{
|
||||
throw_error();
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** Реализация для массивов строк.
|
||||
* NOTE: Код слишком сложный, потому что он работает в внутренностями массивов строк.
|
||||
*/
|
||||
struct StringArrayIfImpl
|
||||
{
|
||||
static ALWAYS_INLINE void copy_from_vector(
|
||||
size_t i,
|
||||
const ColumnString::Chars_t & from_data,
|
||||
const ColumnString::Offsets_t & from_string_offsets,
|
||||
const ColumnArray::Offsets_t & from_array_offsets,
|
||||
const ColumnArray::Offset_t & from_array_prev_offset,
|
||||
const ColumnString::Offset_t & from_string_prev_offset,
|
||||
ColumnString::Chars_t & to_data,
|
||||
ColumnString::Offsets_t & to_string_offsets,
|
||||
ColumnArray::Offsets_t & to_array_offsets,
|
||||
ColumnArray::Offset_t & to_array_prev_offset,
|
||||
ColumnString::Offset_t & to_string_prev_offset)
|
||||
{
|
||||
size_t array_size = from_array_offsets[i] - from_array_prev_offset;
|
||||
|
||||
size_t bytes_to_copy = 0;
|
||||
size_t from_string_prev_offset_local = from_string_prev_offset;
|
||||
for (size_t j = 0; j < array_size; ++j)
|
||||
{
|
||||
size_t string_size = from_string_offsets[from_array_prev_offset + j] - from_string_prev_offset_local;
|
||||
|
||||
to_string_prev_offset += string_size;
|
||||
to_string_offsets.push_back(to_string_prev_offset);
|
||||
|
||||
from_string_prev_offset_local += string_size;
|
||||
bytes_to_copy += string_size;
|
||||
}
|
||||
|
||||
size_t to_data_old_size = to_data.size();
|
||||
to_data.resize(to_data_old_size + bytes_to_copy);
|
||||
memcpy(&to_data[to_data_old_size], &from_data[from_string_prev_offset], bytes_to_copy);
|
||||
|
||||
to_array_prev_offset += array_size;
|
||||
to_array_offsets[i] = to_array_prev_offset;
|
||||
}
|
||||
|
||||
static ALWAYS_INLINE void copy_from_constant(
|
||||
size_t i,
|
||||
const Array & from_data,
|
||||
ColumnString::Chars_t & to_data,
|
||||
ColumnString::Offsets_t & to_string_offsets,
|
||||
ColumnArray::Offsets_t & to_array_offsets,
|
||||
ColumnArray::Offset_t & to_array_prev_offset,
|
||||
ColumnString::Offset_t & to_string_prev_offset)
|
||||
{
|
||||
size_t array_size = from_data.size();
|
||||
|
||||
for (size_t j = 0; j < array_size; ++j)
|
||||
{
|
||||
const String & str = from_data[j].get<const String &>();
|
||||
size_t string_size = str.size() + 1; /// Включая 0 на конце.
|
||||
|
||||
to_data.resize(to_string_prev_offset + string_size);
|
||||
memcpy(&to_data[to_string_prev_offset], str.data(), string_size);
|
||||
|
||||
to_string_prev_offset += string_size;
|
||||
to_string_offsets.push_back(to_string_prev_offset);
|
||||
}
|
||||
|
||||
to_array_prev_offset += array_size;
|
||||
to_array_offsets[i] = to_array_prev_offset;
|
||||
}
|
||||
|
||||
|
||||
static void vector_vector(
|
||||
const PODArray<UInt8> & cond,
|
||||
const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_string_offsets, const ColumnArray::Offsets_t & a_array_offsets,
|
||||
const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_string_offsets, const ColumnArray::Offsets_t & b_array_offsets,
|
||||
ColumnString::Chars_t & c_data, ColumnString::Offsets_t & c_string_offsets, ColumnArray::Offsets_t & c_array_offsets)
|
||||
{
|
||||
size_t size = cond.size();
|
||||
c_array_offsets.resize(size);
|
||||
c_string_offsets.reserve(std::max(a_string_offsets.size(), b_string_offsets.size()));
|
||||
c_data.reserve(std::max(a_data.size(), b_data.size()));
|
||||
|
||||
ColumnArray::Offset_t a_array_prev_offset = 0;
|
||||
ColumnArray::Offset_t b_array_prev_offset = 0;
|
||||
ColumnArray::Offset_t c_array_prev_offset = 0;
|
||||
|
||||
ColumnString::Offset_t a_string_prev_offset = 0;
|
||||
ColumnString::Offset_t b_string_prev_offset = 0;
|
||||
ColumnString::Offset_t c_string_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (cond[i])
|
||||
copy_from_vector(i,
|
||||
a_data, a_string_offsets, a_array_offsets, a_array_prev_offset, a_string_prev_offset,
|
||||
c_data, c_string_offsets, c_array_offsets, c_array_prev_offset, c_string_prev_offset);
|
||||
else
|
||||
copy_from_vector(i,
|
||||
b_data, b_string_offsets, b_array_offsets, b_array_prev_offset, b_string_prev_offset,
|
||||
c_data, c_string_offsets, c_array_offsets, c_array_prev_offset, c_string_prev_offset);
|
||||
|
||||
a_array_prev_offset = a_array_offsets[i];
|
||||
b_array_prev_offset = b_array_offsets[i];
|
||||
|
||||
if (a_array_prev_offset)
|
||||
a_string_prev_offset = a_string_offsets[a_array_prev_offset - 1];
|
||||
|
||||
if (b_array_prev_offset)
|
||||
b_string_prev_offset = b_string_offsets[b_array_prev_offset - 1];
|
||||
}
|
||||
}
|
||||
|
||||
template <bool reverse>
|
||||
static void vector_constant_impl(
|
||||
const PODArray<UInt8> & cond,
|
||||
const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_string_offsets, const ColumnArray::Offsets_t & a_array_offsets,
|
||||
const Array & b,
|
||||
ColumnString::Chars_t & c_data, ColumnString::Offsets_t & c_string_offsets, ColumnArray::Offsets_t & c_array_offsets)
|
||||
{
|
||||
size_t size = cond.size();
|
||||
c_array_offsets.resize(size);
|
||||
c_string_offsets.reserve(a_string_offsets.size());
|
||||
c_data.reserve(a_data.size());
|
||||
|
||||
ColumnArray::Offset_t a_array_prev_offset = 0;
|
||||
ColumnArray::Offset_t c_array_prev_offset = 0;
|
||||
|
||||
ColumnString::Offset_t a_string_prev_offset = 0;
|
||||
ColumnString::Offset_t c_string_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (reverse != cond[i])
|
||||
copy_from_vector(i,
|
||||
a_data, a_string_offsets, a_array_offsets, a_array_prev_offset, a_string_prev_offset,
|
||||
c_data, c_string_offsets, c_array_offsets, c_array_prev_offset, c_string_prev_offset);
|
||||
else
|
||||
copy_from_constant(i,
|
||||
b,
|
||||
c_data, c_string_offsets, c_array_offsets, c_array_prev_offset, c_string_prev_offset);
|
||||
|
||||
a_array_prev_offset = a_array_offsets[i];
|
||||
|
||||
if (a_array_prev_offset)
|
||||
a_string_prev_offset = a_string_offsets[a_array_prev_offset - 1];
|
||||
}
|
||||
}
|
||||
|
||||
static void vector_constant(
|
||||
const PODArray<UInt8> & cond,
|
||||
const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_string_offsets, const ColumnArray::Offsets_t & a_array_offsets,
|
||||
const Array & b,
|
||||
ColumnString::Chars_t & c_data, ColumnString::Offsets_t & c_string_offsets, ColumnArray::Offsets_t & c_array_offsets)
|
||||
{
|
||||
vector_constant_impl<false>(cond, a_data, a_string_offsets, a_array_offsets, b, c_data, c_string_offsets, c_array_offsets);
|
||||
}
|
||||
|
||||
static void constant_vector(
|
||||
const PODArray<UInt8> & cond,
|
||||
const Array & a,
|
||||
const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_string_offsets, const ColumnArray::Offsets_t & b_array_offsets,
|
||||
ColumnString::Chars_t & c_data, ColumnString::Offsets_t & c_string_offsets, ColumnArray::Offsets_t & c_array_offsets)
|
||||
{
|
||||
vector_constant_impl<true>(cond, b_data, b_string_offsets, b_array_offsets, a, c_data, c_string_offsets, c_array_offsets);
|
||||
}
|
||||
|
||||
static void constant_constant(
|
||||
const PODArray<UInt8> & cond,
|
||||
const Array & a,
|
||||
const Array & b,
|
||||
ColumnString::Chars_t & c_data, ColumnString::Offsets_t & c_string_offsets, ColumnArray::Offsets_t & c_array_offsets)
|
||||
{
|
||||
size_t size = cond.size();
|
||||
c_array_offsets.resize(size);
|
||||
c_string_offsets.reserve(std::max(a.size(), b.size()) * size);
|
||||
|
||||
size_t sum_size_a = 0;
|
||||
for (const auto & s : a)
|
||||
sum_size_a += s.get<const String &>().size() + 1;
|
||||
|
||||
size_t sum_size_b = 0;
|
||||
for (const auto & s : b)
|
||||
sum_size_b += s.get<const String &>().size() + 1;
|
||||
|
||||
c_data.reserve(std::max(sum_size_a, sum_size_b) * size);
|
||||
|
||||
ColumnArray::Offset_t c_array_prev_offset = 0;
|
||||
ColumnString::Offset_t c_string_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (cond[i])
|
||||
copy_from_constant(i,
|
||||
a,
|
||||
c_data, c_string_offsets, c_array_offsets, c_array_prev_offset, c_string_prev_offset);
|
||||
else
|
||||
copy_from_constant(i,
|
||||
b,
|
||||
c_data, c_string_offsets, c_array_offsets, c_array_prev_offset, c_string_prev_offset);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <typename T>
|
||||
struct DataTypeFromFieldTypeOrError
|
||||
{
|
||||
@ -347,8 +767,8 @@ private:
|
||||
size_t result,
|
||||
const ColumnVector<T0> * col_left)
|
||||
{
|
||||
ColumnVector<T1> * col_right_vec = typeid_cast<ColumnVector<T1> *>(&*block.getByPosition(arguments[2]).column);
|
||||
ColumnConst<T1> * col_right_const = typeid_cast<ColumnConst<T1> *>(&*block.getByPosition(arguments[2]).column);
|
||||
const ColumnVector<T1> * col_right_vec = typeid_cast<const ColumnVector<T1> *>(&*block.getByPosition(arguments[2]).column);
|
||||
const ColumnConst<T1> * col_right_const = typeid_cast<const ColumnConst<T1> *>(&*block.getByPosition(arguments[2]).column);
|
||||
|
||||
if (!col_right_vec && !col_right_const)
|
||||
return false;
|
||||
@ -371,8 +791,8 @@ private:
|
||||
size_t result,
|
||||
const ColumnConst<T0> * col_left)
|
||||
{
|
||||
ColumnVector<T1> * col_right_vec = typeid_cast<ColumnVector<T1> *>(&*block.getByPosition(arguments[2]).column);
|
||||
ColumnConst<T1> * col_right_const = typeid_cast<ColumnConst<T1> *>(&*block.getByPosition(arguments[2]).column);
|
||||
const ColumnVector<T1> * col_right_vec = typeid_cast<const ColumnVector<T1> *>(&*block.getByPosition(arguments[2]).column);
|
||||
const ColumnConst<T1> * col_right_const = typeid_cast<const ColumnConst<T1> *>(&*block.getByPosition(arguments[2]).column);
|
||||
|
||||
if (!col_right_vec && !col_right_const)
|
||||
return false;
|
||||
@ -387,10 +807,128 @@ private:
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename T0, typename T1>
|
||||
bool executeRightTypeArray(
|
||||
const ColumnVector<UInt8> * cond_col,
|
||||
Block & block,
|
||||
const ColumnNumbers & arguments,
|
||||
size_t result,
|
||||
const ColumnArray * col_left_array,
|
||||
const ColumnVector<T0> * col_left)
|
||||
{
|
||||
const IColumn * col_right_untyped = block.getByPosition(arguments[2]).column.get();
|
||||
|
||||
const ColumnArray * col_right_array = typeid_cast<const ColumnArray *>(col_right_untyped);
|
||||
const ColumnConstArray * col_right_const_array = typeid_cast<const ColumnConstArray *>(col_right_untyped);
|
||||
|
||||
if (!col_right_array && !col_right_const_array)
|
||||
return false;
|
||||
|
||||
typedef typename NumberTraits::ResultOfIf<T0, T1>::Type ResultType;
|
||||
|
||||
if (col_right_array)
|
||||
{
|
||||
const ColumnVector<T1> * col_right_vec = typeid_cast<const ColumnVector<T1> *>(&col_right_array->getData());
|
||||
|
||||
if (!col_right_vec)
|
||||
return false;
|
||||
|
||||
NumArrayIfImpl<T0, T1, ResultType>::vector_vector(
|
||||
cond_col->getData(),
|
||||
col_left->getData(), col_left_array->getOffsets(),
|
||||
col_right_vec->getData(), col_right_array->getOffsets(),
|
||||
block, result);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!typeid_cast<const typename DataTypeFromFieldType<T1>::Type *>(
|
||||
typeid_cast<const DataTypeArray &>(*col_right_const_array->getDataType()).getNestedType().get()))
|
||||
return false;
|
||||
|
||||
NumArrayIfImpl<T0, T1, ResultType>::vector_constant(
|
||||
cond_col->getData(),
|
||||
col_left->getData(), col_left_array->getOffsets(),
|
||||
col_right_const_array->getData(),
|
||||
block, result);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename T0, typename T1>
|
||||
bool executeConstRightTypeArray(
|
||||
const ColumnVector<UInt8> * cond_col,
|
||||
Block & block,
|
||||
const ColumnNumbers & arguments,
|
||||
size_t result,
|
||||
const ColumnConstArray * col_left_const_array)
|
||||
{
|
||||
const IColumn * col_right_untyped = block.getByPosition(arguments[2]).column.get();
|
||||
|
||||
const ColumnArray * col_right_array = typeid_cast<const ColumnArray *>(col_right_untyped);
|
||||
const ColumnConstArray * col_right_const_array = typeid_cast<const ColumnConstArray *>(col_right_untyped);
|
||||
|
||||
if (!col_right_array && !col_right_const_array)
|
||||
return false;
|
||||
|
||||
typedef typename NumberTraits::ResultOfIf<T0, T1>::Type ResultType;
|
||||
|
||||
if (col_right_array)
|
||||
{
|
||||
const ColumnVector<T1> * col_right_vec = typeid_cast<const ColumnVector<T1> *>(&col_right_array->getData());
|
||||
|
||||
if (!col_right_vec)
|
||||
return false;
|
||||
|
||||
NumArrayIfImpl<T0, T1, ResultType>::constant_vector(
|
||||
cond_col->getData(),
|
||||
col_left_const_array->getData(),
|
||||
col_right_vec->getData(), col_right_array->getOffsets(),
|
||||
block, result);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!typeid_cast<const typename DataTypeFromFieldType<T1>::Type *>(
|
||||
typeid_cast<const DataTypeArray &>(*col_right_const_array->getDataType()).getNestedType().get()))
|
||||
return false;
|
||||
|
||||
NumArrayIfImpl<T0, T1, ResultType>::constant_constant(
|
||||
cond_col->getData(),
|
||||
col_left_const_array->getData(),
|
||||
col_right_const_array->getData(),
|
||||
block, result);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename T0>
|
||||
bool executeLeftType(const ColumnVector<UInt8> * cond_col, Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (ColumnVector<T0> * col_left = typeid_cast<ColumnVector<T0> *>(&*block.getByPosition(arguments[1]).column))
|
||||
const IColumn * col_left_untyped = block.getByPosition(arguments[1]).column.get();
|
||||
|
||||
const ColumnVector<T0> * col_left = nullptr;
|
||||
const ColumnConst<T0> * col_const_left = nullptr;
|
||||
const ColumnArray * col_arr_left = nullptr;
|
||||
const ColumnVector<T0> * col_arr_left_elems = nullptr;
|
||||
const ColumnConstArray * col_const_arr_left = nullptr;
|
||||
|
||||
col_left = typeid_cast<const ColumnVector<T0> *>(col_left_untyped);
|
||||
if (!col_left)
|
||||
{
|
||||
col_const_left = typeid_cast<const ColumnConst<T0> *>(col_left_untyped);
|
||||
if (!col_const_left)
|
||||
{
|
||||
col_arr_left = typeid_cast<const ColumnArray *>(col_left_untyped);
|
||||
|
||||
if (col_arr_left)
|
||||
col_arr_left_elems = typeid_cast<const ColumnVector<T0> *>(&col_arr_left->getData());
|
||||
else
|
||||
col_const_arr_left = typeid_cast<const ColumnConstArray *>(col_left_untyped);
|
||||
}
|
||||
}
|
||||
|
||||
if (col_left)
|
||||
{
|
||||
if ( executeRightType<T0, UInt8>(cond_col, block, arguments, result, col_left)
|
||||
|| executeRightType<T0, UInt16>(cond_col, block, arguments, result, col_left)
|
||||
@ -408,18 +946,56 @@ private:
|
||||
+ " of third argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (ColumnConst<T0> * col_left = typeid_cast<ColumnConst<T0> *>(&*block.getByPosition(arguments[1]).column))
|
||||
else if (col_const_left)
|
||||
{
|
||||
if ( executeConstRightType<T0, UInt8>(cond_col, block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, UInt16>(cond_col, block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, UInt32>(cond_col, block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, UInt64>(cond_col, block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Int8>(cond_col, block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Int16>(cond_col, block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Int32>(cond_col, block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Int64>(cond_col, block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Float32>(cond_col, block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Float64>(cond_col, block, arguments, result, col_left))
|
||||
if ( executeConstRightType<T0, UInt8>(cond_col, block, arguments, result, col_const_left)
|
||||
|| executeConstRightType<T0, UInt16>(cond_col, block, arguments, result, col_const_left)
|
||||
|| executeConstRightType<T0, UInt32>(cond_col, block, arguments, result, col_const_left)
|
||||
|| executeConstRightType<T0, UInt64>(cond_col, block, arguments, result, col_const_left)
|
||||
|| executeConstRightType<T0, Int8>(cond_col, block, arguments, result, col_const_left)
|
||||
|| executeConstRightType<T0, Int16>(cond_col, block, arguments, result, col_const_left)
|
||||
|| executeConstRightType<T0, Int32>(cond_col, block, arguments, result, col_const_left)
|
||||
|| executeConstRightType<T0, Int64>(cond_col, block, arguments, result, col_const_left)
|
||||
|| executeConstRightType<T0, Float32>(cond_col, block, arguments, result, col_const_left)
|
||||
|| executeConstRightType<T0, Float64>(cond_col, block, arguments, result, col_const_left))
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[2]).column->getName()
|
||||
+ " of third argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (col_arr_left && col_arr_left_elems)
|
||||
{
|
||||
if ( executeRightTypeArray<T0, UInt8>(cond_col, block, arguments, result, col_arr_left, col_arr_left_elems)
|
||||
|| executeRightTypeArray<T0, UInt16>(cond_col, block, arguments, result, col_arr_left, col_arr_left_elems)
|
||||
|| executeRightTypeArray<T0, UInt32>(cond_col, block, arguments, result, col_arr_left, col_arr_left_elems)
|
||||
|| executeRightTypeArray<T0, UInt64>(cond_col, block, arguments, result, col_arr_left, col_arr_left_elems)
|
||||
|| executeRightTypeArray<T0, Int8>(cond_col, block, arguments, result, col_arr_left, col_arr_left_elems)
|
||||
|| executeRightTypeArray<T0, Int16>(cond_col, block, arguments, result, col_arr_left, col_arr_left_elems)
|
||||
|| executeRightTypeArray<T0, Int32>(cond_col, block, arguments, result, col_arr_left, col_arr_left_elems)
|
||||
|| executeRightTypeArray<T0, Int64>(cond_col, block, arguments, result, col_arr_left, col_arr_left_elems)
|
||||
|| executeRightTypeArray<T0, Float32>(cond_col, block, arguments, result, col_arr_left, col_arr_left_elems)
|
||||
|| executeRightTypeArray<T0, Float64>(cond_col, block, arguments, result, col_arr_left, col_arr_left_elems))
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[2]).column->getName()
|
||||
+ " of third argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (col_const_arr_left
|
||||
&& typeid_cast<const typename DataTypeFromFieldType<T0>::Type *>(
|
||||
typeid_cast<const DataTypeArray &>(*col_const_arr_left->getDataType()).getNestedType().get()))
|
||||
{
|
||||
if ( executeConstRightTypeArray<T0, UInt8>(cond_col, block, arguments, result, col_const_arr_left)
|
||||
|| executeConstRightTypeArray<T0, UInt16>(cond_col, block, arguments, result, col_const_arr_left)
|
||||
|| executeConstRightTypeArray<T0, UInt32>(cond_col, block, arguments, result, col_const_arr_left)
|
||||
|| executeConstRightTypeArray<T0, UInt64>(cond_col, block, arguments, result, col_const_arr_left)
|
||||
|| executeConstRightTypeArray<T0, Int8>(cond_col, block, arguments, result, col_const_arr_left)
|
||||
|| executeConstRightTypeArray<T0, Int16>(cond_col, block, arguments, result, col_const_arr_left)
|
||||
|| executeConstRightTypeArray<T0, Int32>(cond_col, block, arguments, result, col_const_arr_left)
|
||||
|| executeConstRightTypeArray<T0, Int64>(cond_col, block, arguments, result, col_const_arr_left)
|
||||
|| executeConstRightTypeArray<T0, Float32>(cond_col, block, arguments, result, col_const_arr_left)
|
||||
|| executeConstRightTypeArray<T0, Float64>(cond_col, block, arguments, result, col_const_arr_left))
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[2]).column->getName()
|
||||
@ -432,11 +1008,16 @@ private:
|
||||
|
||||
bool executeString(const ColumnVector<UInt8> * cond_col, Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
ColumnString * col_then = typeid_cast<ColumnString *>(&*block.getByPosition(arguments[1]).column);
|
||||
ColumnString * col_else = typeid_cast<ColumnString *>(&*block.getByPosition(arguments[2]).column);
|
||||
ColumnConstString * col_then_const = typeid_cast<ColumnConstString *>(&*block.getByPosition(arguments[1]).column);
|
||||
ColumnConstString * col_else_const = typeid_cast<ColumnConstString *>(&*block.getByPosition(arguments[2]).column);
|
||||
const IColumn * col_then_untyped = block.getByPosition(arguments[1]).column.get();
|
||||
const IColumn * col_else_untyped = block.getByPosition(arguments[2]).column.get();
|
||||
|
||||
const ColumnString * col_then = typeid_cast<const ColumnString *>(col_then_untyped);
|
||||
const ColumnString * col_else = typeid_cast<const ColumnString *>(col_else_untyped);
|
||||
const ColumnConstString * col_then_const = typeid_cast<const ColumnConstString *>(col_then_untyped);
|
||||
const ColumnConstString * col_else_const = typeid_cast<const ColumnConstString *>(col_else_untyped);
|
||||
|
||||
if ((col_then || col_then_const) && (col_else || col_else_const))
|
||||
{
|
||||
ColumnString * col_res = new ColumnString;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
@ -473,6 +1054,57 @@ private:
|
||||
return true;
|
||||
}
|
||||
|
||||
const ColumnArray * col_arr_then = typeid_cast<const ColumnArray *>(col_then_untyped);
|
||||
const ColumnArray * col_arr_else = typeid_cast<const ColumnArray *>(col_else_untyped);
|
||||
const ColumnConstArray * col_arr_then_const = typeid_cast<const ColumnConstArray *>(col_then_untyped);
|
||||
const ColumnConstArray * col_arr_else_const = typeid_cast<const ColumnConstArray *>(col_else_untyped);
|
||||
const ColumnString * col_then_elements = col_arr_then ? typeid_cast<const ColumnString *>(&col_arr_then->getData()) : nullptr;
|
||||
const ColumnString * col_else_elements = col_arr_else ? typeid_cast<const ColumnString *>(&col_arr_else->getData()) : nullptr;
|
||||
|
||||
if (((col_arr_then && col_then_elements) || col_arr_then_const)
|
||||
&& ((col_arr_else && col_else_elements) || col_arr_else_const))
|
||||
{
|
||||
ColumnString * col_res_elements = new ColumnString;
|
||||
ColumnArray * col_res = new ColumnArray(col_res_elements);
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
ColumnString::Chars_t & res_chars = col_res_elements->getChars();
|
||||
ColumnString::Offsets_t & res_string_offsets = col_res_elements->getOffsets();
|
||||
ColumnArray::Offsets_t & res_array_offsets = col_res->getOffsets();
|
||||
|
||||
if (col_then_elements && col_else_elements)
|
||||
StringArrayIfImpl::vector_vector(
|
||||
cond_col->getData(),
|
||||
col_then_elements->getChars(), col_then_elements->getOffsets(), col_arr_then->getOffsets(),
|
||||
col_else_elements->getChars(), col_else_elements->getOffsets(), col_arr_else->getOffsets(),
|
||||
res_chars, res_string_offsets, res_array_offsets);
|
||||
else if (col_then_elements && col_arr_else_const)
|
||||
StringArrayIfImpl::vector_constant(
|
||||
cond_col->getData(),
|
||||
col_then_elements->getChars(), col_then_elements->getOffsets(), col_arr_then->getOffsets(),
|
||||
col_arr_else_const->getData(),
|
||||
res_chars, res_string_offsets, res_array_offsets);
|
||||
else if (col_arr_then_const && col_else_elements)
|
||||
StringArrayIfImpl::constant_vector(
|
||||
cond_col->getData(),
|
||||
col_arr_then_const->getData(),
|
||||
col_else_elements->getChars(), col_else_elements->getOffsets(), col_arr_else->getOffsets(),
|
||||
res_chars, res_string_offsets, res_array_offsets);
|
||||
else if (col_arr_then_const && col_arr_else_const)
|
||||
StringArrayIfImpl::constant_constant(
|
||||
cond_col->getData(),
|
||||
col_arr_then_const->getData(),
|
||||
col_arr_else_const->getData(),
|
||||
res_chars, res_string_offsets, res_array_offsets);
|
||||
else
|
||||
return false;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
public:
|
||||
/// Получить имя функции.
|
||||
String getName() const
|
||||
@ -492,6 +1124,9 @@ public:
|
||||
throw Exception("Illegal type of first argument (condition) of function if. Must be UInt8.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const DataTypeArray * type_arr1 = typeid_cast<const DataTypeArray *>(arguments[1].get());
|
||||
const DataTypeArray * type_arr2 = typeid_cast<const DataTypeArray *>(arguments[2].get());
|
||||
|
||||
if (arguments[1]->behavesAsNumber() && arguments[2]->behavesAsNumber())
|
||||
{
|
||||
DataTypePtr type_res;
|
||||
@ -509,6 +1144,11 @@ public:
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
return type_res;
|
||||
}
|
||||
else if (type_arr1 && type_arr2)
|
||||
{
|
||||
/// NOTE Сообщения об ошибках будут относится к типам элементов массивов, что немного некорректно.
|
||||
return new DataTypeArray(getReturnType({arguments[0], type_arr1->getNestedType(), type_arr2->getNestedType()}));
|
||||
}
|
||||
else if (arguments[1]->getName() != arguments[2]->getName())
|
||||
{
|
||||
throw Exception("Incompatible second and third arguments for function " + getName() + ": "
|
||||
@ -542,6 +1182,7 @@ public:
|
||||
cond_col = typeid_cast<const ColumnVector<UInt8> *>(&*materialized_cond_col);
|
||||
}
|
||||
}
|
||||
|
||||
if (cond_col)
|
||||
{
|
||||
if (!( executeLeftType<UInt8>(cond_col, block, arguments, result)
|
||||
@ -561,7 +1202,8 @@ public:
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + cond_col->getName() + " of first argument of function " + getName() + ". Must be ColumnUInt8 or ColumnConstUInt8.",
|
||||
throw Exception("Illegal column " + cond_col->getName() + " of first argument of function " + getName()
|
||||
+ ". Must be ColumnUInt8 or ColumnConstUInt8.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
|
||||
#include <DB/Columns/ColumnArray.h>
|
||||
#include <DB/Columns/ColumnReplicated.h>
|
||||
#include <DB/Columns/ColumnExpression.h>
|
||||
|
||||
#include <DB/Functions/IFunction.h>
|
||||
@ -42,7 +41,9 @@ struct ArrayMapImpl
|
||||
|
||||
static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped)
|
||||
{
|
||||
return new ColumnArray(mapped, array->getOffsetsColumn());
|
||||
return mapped->isConst()
|
||||
? new ColumnArray(dynamic_cast<const IColumnConst &>(*mapped).convertToFullColumn(), array->getOffsetsColumn())
|
||||
: new ColumnArray(mapped, array->getOffsetsColumn());
|
||||
}
|
||||
};
|
||||
|
||||
@ -60,10 +61,21 @@ struct ArrayFilterImpl
|
||||
/// Если массивов несколько, сюда передается первый.
|
||||
static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped)
|
||||
{
|
||||
ColumnVector<UInt8> * column_filter = typeid_cast<ColumnVector<UInt8> *>(&*mapped);
|
||||
const ColumnVector<UInt8> * column_filter = typeid_cast<const ColumnVector<UInt8> *>(&*mapped);
|
||||
|
||||
if (!column_filter)
|
||||
{
|
||||
const ColumnConstUInt8 * column_filter_const = typeid_cast<const ColumnConstUInt8 *>(&*mapped);
|
||||
|
||||
if (!column_filter_const)
|
||||
throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if (column_filter_const->getData())
|
||||
return array->clone();
|
||||
else
|
||||
return new ColumnArray(array->getDataPtr()->cloneEmpty(), new ColumnArray::ColumnOffsets_t(array->size(), 0));
|
||||
}
|
||||
|
||||
const IColumn::Filter & filter = column_filter->getData();
|
||||
ColumnPtr filtered = array->getData().filter(filter);
|
||||
|
||||
@ -101,10 +113,35 @@ struct ArrayCountImpl
|
||||
|
||||
static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped)
|
||||
{
|
||||
ColumnVector<UInt8> * column_filter = typeid_cast<ColumnVector<UInt8> *>(&*mapped);
|
||||
const ColumnVector<UInt8> * column_filter = typeid_cast<const ColumnVector<UInt8> *>(&*mapped);
|
||||
|
||||
if (!column_filter)
|
||||
{
|
||||
const ColumnConstUInt8 * column_filter_const = typeid_cast<const ColumnConstUInt8 *>(&*mapped);
|
||||
|
||||
if (!column_filter_const)
|
||||
throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if (column_filter_const->getData())
|
||||
{
|
||||
const IColumn::Offsets_t & offsets = array->getOffsets();
|
||||
ColumnVector<UInt32> * out_column = new ColumnVector<UInt32>(offsets.size());
|
||||
ColumnPtr out_column_ptr = out_column;
|
||||
ColumnVector<UInt32>::Container_t & out_counts = out_column->getData();
|
||||
|
||||
size_t pos = 0;
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
out_counts[i] = offsets[i] - pos;
|
||||
pos = offsets[i];
|
||||
}
|
||||
|
||||
return out_column_ptr;
|
||||
}
|
||||
else
|
||||
return new ColumnConstUInt32(array->size(), 0);
|
||||
}
|
||||
|
||||
const IColumn::Filter & filter = column_filter->getData();
|
||||
const IColumn::Offsets_t & offsets = array->getOffsets();
|
||||
ColumnVector<UInt32> * out_column = new ColumnVector<UInt32>(offsets.size());
|
||||
@ -140,10 +177,35 @@ struct ArrayExistsImpl
|
||||
|
||||
static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped)
|
||||
{
|
||||
ColumnVector<UInt8> * column_filter = typeid_cast<ColumnVector<UInt8> *>(&*mapped);
|
||||
const ColumnVector<UInt8> * column_filter = typeid_cast<const ColumnVector<UInt8> *>(&*mapped);
|
||||
|
||||
if (!column_filter)
|
||||
{
|
||||
const ColumnConstUInt8 * column_filter_const = typeid_cast<const ColumnConstUInt8 *>(&*mapped);
|
||||
|
||||
if (!column_filter_const)
|
||||
throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if (column_filter_const->getData())
|
||||
{
|
||||
const IColumn::Offsets_t & offsets = array->getOffsets();
|
||||
ColumnVector<UInt8> * out_column = new ColumnVector<UInt8>(offsets.size());
|
||||
ColumnPtr out_column_ptr = out_column;
|
||||
ColumnVector<UInt8>::Container_t & out_exists = out_column->getData();
|
||||
|
||||
size_t pos = 0;
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
out_exists[i] = offsets[i] - pos > 0;
|
||||
pos = offsets[i];
|
||||
}
|
||||
|
||||
return out_column_ptr;
|
||||
}
|
||||
else
|
||||
return new ColumnConstUInt8(array->size(), 0);
|
||||
}
|
||||
|
||||
const IColumn::Filter & filter = column_filter->getData();
|
||||
const IColumn::Offsets_t & offsets = array->getOffsets();
|
||||
ColumnVector<UInt8> * out_column = new ColumnVector<UInt8>(offsets.size());
|
||||
@ -183,10 +245,35 @@ struct ArrayAllImpl
|
||||
|
||||
static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped)
|
||||
{
|
||||
ColumnVector<UInt8> * column_filter = typeid_cast<ColumnVector<UInt8> *>(&*mapped);
|
||||
const ColumnVector<UInt8> * column_filter = typeid_cast<const ColumnVector<UInt8> *>(&*mapped);
|
||||
|
||||
if (!column_filter)
|
||||
{
|
||||
const ColumnConstUInt8 * column_filter_const = typeid_cast<const ColumnConstUInt8 *>(&*mapped);
|
||||
|
||||
if (!column_filter_const)
|
||||
throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if (column_filter_const->getData())
|
||||
return new ColumnConstUInt8(array->size(), 1);
|
||||
else
|
||||
{
|
||||
const IColumn::Offsets_t & offsets = array->getOffsets();
|
||||
ColumnVector<UInt8> * out_column = new ColumnVector<UInt8>(offsets.size());
|
||||
ColumnPtr out_column_ptr = out_column;
|
||||
ColumnVector<UInt8>::Container_t & out_all = out_column->getData();
|
||||
|
||||
size_t pos = 0;
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
out_all[i] = offsets[i] == pos;
|
||||
pos = offsets[i];
|
||||
}
|
||||
|
||||
return out_column_ptr;
|
||||
}
|
||||
}
|
||||
|
||||
const IColumn::Filter & filter = column_filter->getData();
|
||||
const IColumn::Offsets_t & offsets = array->getOffsets();
|
||||
ColumnVector<UInt8> * out_column = new ColumnVector<UInt8>(offsets.size());
|
||||
@ -246,8 +333,28 @@ struct ArraySumImpl
|
||||
const ColumnVector<Element> * column = typeid_cast<const ColumnVector<Element> *>(&*mapped);
|
||||
|
||||
if (!column)
|
||||
{
|
||||
const ColumnConst<Element> * column_const = typeid_cast<const ColumnConst<Element> *>(&*mapped);
|
||||
|
||||
if (!column_const)
|
||||
return false;
|
||||
|
||||
const Element x = column_const->getData();
|
||||
|
||||
ColumnVector<Result> * res_column = new ColumnVector<Result>(offsets.size());
|
||||
res_ptr = res_column;
|
||||
typename ColumnVector<Result>::Container_t & res = res_column->getData();
|
||||
|
||||
size_t pos = 0;
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
res[i] = x * (offsets[i] - pos);
|
||||
pos = offsets[i];
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
const typename ColumnVector<Element>::Container_t & data = column->getData();
|
||||
ColumnVector<Result> * res_column = new ColumnVector<Result>(offsets.size());
|
||||
res_ptr = res_column;
|
||||
@ -301,10 +408,42 @@ struct ArrayFirstImpl
|
||||
|
||||
static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped)
|
||||
{
|
||||
auto column_filter = typeid_cast<ColumnVector<UInt8> *>(&*mapped);
|
||||
auto column_filter = typeid_cast<const ColumnVector<UInt8> *>(&*mapped);
|
||||
|
||||
if (!column_filter)
|
||||
{
|
||||
const ColumnConstUInt8 * column_filter_const = typeid_cast<const ColumnConstUInt8 *>(&*mapped);
|
||||
|
||||
if (!column_filter_const)
|
||||
throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if (column_filter_const->getData())
|
||||
{
|
||||
const auto & offsets = array->getOffsets();
|
||||
const auto & data = array->getData();
|
||||
ColumnPtr out{data.cloneEmpty()};
|
||||
|
||||
size_t pos{};
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
if (offsets[i] - pos > 0)
|
||||
out->insert(data[pos]);
|
||||
else
|
||||
out->insertDefault();
|
||||
|
||||
pos = offsets[i];
|
||||
}
|
||||
|
||||
return out;
|
||||
}
|
||||
else
|
||||
{
|
||||
ColumnPtr out{array->getData().cloneEmpty()};
|
||||
out->insertDefault();
|
||||
return out->replicate(IColumn::Offsets_t(1, array->size()));
|
||||
}
|
||||
}
|
||||
|
||||
const auto & filter = column_filter->getData();
|
||||
const auto & offsets = array->getOffsets();
|
||||
const auto & data = array->getData();
|
||||
@ -346,10 +485,35 @@ struct ArrayFirstIndexImpl
|
||||
|
||||
static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped)
|
||||
{
|
||||
auto column_filter = typeid_cast<ColumnVector<UInt8> *>(&*mapped);
|
||||
auto column_filter = typeid_cast<const ColumnVector<UInt8> *>(&*mapped);
|
||||
|
||||
if (!column_filter)
|
||||
{
|
||||
const ColumnConstUInt8 * column_filter_const = typeid_cast<const ColumnConstUInt8 *>(&*mapped);
|
||||
|
||||
if (!column_filter_const)
|
||||
throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if (column_filter_const->getData())
|
||||
{
|
||||
const auto & offsets = array->getOffsets();
|
||||
auto out_column = new ColumnVector<UInt32>{offsets.size()};
|
||||
ColumnPtr out_column_ptr{out_column};
|
||||
auto & out_index = out_column->getData();
|
||||
|
||||
size_t pos{};
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
out_index[i] = offsets[i] - pos > 0;
|
||||
pos = offsets[i];
|
||||
}
|
||||
|
||||
return out_column_ptr;
|
||||
}
|
||||
else
|
||||
return new ColumnConstUInt32(array->size(), 0);
|
||||
}
|
||||
|
||||
const auto & filter = column_filter->getData();
|
||||
const auto & offsets = array->getOffsets();
|
||||
auto out_column = new ColumnVector<UInt32>{offsets.size()};
|
||||
@ -580,7 +744,7 @@ public:
|
||||
ColumnWithNameAndType replicated_column = block.getByPosition(prerequisites[prerequisite_index]);
|
||||
|
||||
replicated_column.name = name;
|
||||
replicated_column.column = typeid_cast<ColumnReplicated &>(*replicated_column.column).getData();
|
||||
replicated_column.column = typeid_cast<ColumnArray &>(*replicated_column.column).getDataPtr();
|
||||
temp_block.insert(replicated_column);
|
||||
|
||||
++prerequisite_index;
|
||||
|
@ -20,7 +20,6 @@
|
||||
#include <DB/Columns/ColumnSet.h>
|
||||
#include <DB/Columns/ColumnTuple.h>
|
||||
#include <DB/Columns/ColumnArray.h>
|
||||
#include <DB/Columns/ColumnReplicated.h>
|
||||
#include <DB/Columns/ColumnAggregateFunction.h>
|
||||
#include <DB/Common/UnicodeBar.h>
|
||||
#include <DB/Functions/IFunction.h>
|
||||
@ -51,9 +50,8 @@ namespace DB
|
||||
* arrayJoin(arr) - особая функция - выполнить её напрямую нельзя;
|
||||
* используется только чтобы получить тип результата соответствующего выражения.
|
||||
*
|
||||
* replicate(x, arr) - копирует x столько раз, сколько элементов в массиве arr;
|
||||
* например: replicate(1, ['a', 'b', 'c']) = 1, 1, 1.
|
||||
* не предназначена для пользователя, а используется только как prerequisites для функций высшего порядка.
|
||||
* replicate(x, arr) - создаёт массив такого же размера как arr, все элементы которого равны x;
|
||||
* например: replicate(1, ['a', 'b', 'c']) = [1, 1, 1].
|
||||
*
|
||||
* sleep(n) - спит n секунд каждый блок.
|
||||
*
|
||||
@ -570,18 +568,15 @@ public:
|
||||
};
|
||||
|
||||
|
||||
/** Размножает столбец (первый аргумент) по количеству элементов в массиве (втором аргументе).
|
||||
* Не предназначена для внешнего использования.
|
||||
* Так как возвращаемый столбец будет иметь несовпадающий размер с исходными,
|
||||
* то результат не может быть потом использован в том же блоке, что и аргументы.
|
||||
/** Создаёт массив, размножая столбец (первый аргумент) по количеству элементов в массиве (втором аргументе).
|
||||
* Используется только в качестве prerequisites для функций высшего порядка.
|
||||
*/
|
||||
class FunctionReplicate : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "replicate";
|
||||
static IFunction * create(const Context & context) { return new FunctionReplicate; }
|
||||
|
||||
|
||||
/// Получить имя функции.
|
||||
String getName() const
|
||||
{
|
||||
@ -600,7 +595,7 @@ class FunctionReplicate : public IFunction
|
||||
if (!array_type)
|
||||
throw Exception("Second argument for function " + getName() + " must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return arguments[0]->clone();
|
||||
return new DataTypeArray(arguments[0]->clone());
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
@ -620,7 +615,9 @@ class FunctionReplicate : public IFunction
|
||||
array_column = typeid_cast<ColumnArray *>(&*temp_column);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = new ColumnReplicated(first_column->size(), first_column->replicate(array_column->getOffsets()));
|
||||
block.getByPosition(result).column = new ColumnArray(
|
||||
first_column->replicate(array_column->getOffsets()),
|
||||
array_column->getOffsetsColumn());
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -89,11 +89,12 @@ namespace DB
|
||||
|
||||
static inline ResultType apply(A x)
|
||||
{
|
||||
return x < 18 ? 0
|
||||
return x < 1 ? 0
|
||||
: (x < 18 ? 17
|
||||
: (x < 25 ? 18
|
||||
: (x < 35 ? 25
|
||||
: (x < 45 ? 35
|
||||
: 45)));
|
||||
: 45))));
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -447,10 +447,15 @@ struct MatchImpl
|
||||
const char * str_data = reinterpret_cast<const char *>(&data[i != 0 ? offsets[i - 1] : 0]);
|
||||
size_t str_size = (i != 0 ? offsets[i] - offsets[i - 1] : offsets[0]) - 1;
|
||||
|
||||
/** Даже в случае required_substring_is_prefix используем UNANCHORED проверку регекспа,
|
||||
* чтобы он мог сматчиться, когда required_substring встречается в строке несколько раз,
|
||||
* и на первом вхождении регексп не матчит.
|
||||
*/
|
||||
|
||||
if (required_substring_is_prefix)
|
||||
res[i] = revert ^ regexp->getRE2()->Match(
|
||||
re2_st::StringPiece(str_data, str_size),
|
||||
reinterpret_cast<const char *>(pos) - str_data, str_size, re2_st::RE2::ANCHOR_START, nullptr, 0);
|
||||
reinterpret_cast<const char *>(pos) - str_data, str_size, re2_st::RE2::UNANCHORED, nullptr, 0);
|
||||
else
|
||||
res[i] = revert ^ regexp->getRE2()->Match(
|
||||
re2_st::StringPiece(str_data, str_size),
|
||||
|
@ -84,7 +84,7 @@ public:
|
||||
|
||||
void ignore(size_t n)
|
||||
{
|
||||
while (!eof() && n != 0)
|
||||
while (n != 0 && !eof())
|
||||
{
|
||||
size_t bytes_to_ignore = std::min(static_cast<size_t>(working_buffer.end() - pos), n);
|
||||
pos += bytes_to_ignore;
|
||||
|
@ -105,6 +105,7 @@ void readVectorBinary(std::vector<T> & v, ReadBuffer & buf, size_t MAX_VECTOR_SI
|
||||
|
||||
void assertString(const char * s, ReadBuffer & buf);
|
||||
void assertEOF(ReadBuffer & buf);
|
||||
void assertChar(char symbol, ReadBuffer & buf);
|
||||
|
||||
inline void assertString(const String & s, ReadBuffer & buf)
|
||||
{
|
||||
|
@ -252,13 +252,19 @@ void writeAnyEscapedString(const String & s, WriteBuffer & buf)
|
||||
}
|
||||
|
||||
|
||||
inline void writeEscapedString(const String & s, WriteBuffer & buf)
|
||||
inline void writeEscapedString(const char * str, size_t size, WriteBuffer & buf)
|
||||
{
|
||||
/// strpbrk в libc под Linux на процессорах с SSE 4.2 хорошо оптимизирована (этот if ускоряет код в 1.5 раза)
|
||||
if (nullptr == strpbrk(s.data(), "\b\f\n\r\t\'\\") && strlen(s.data()) == s.size())
|
||||
writeString(s, buf);
|
||||
if (nullptr == strpbrk(str, "\b\f\n\r\t\'\\") && strlen(str) == size)
|
||||
writeString(str, size, buf);
|
||||
else
|
||||
writeAnyEscapedString<'\''>(s, buf);
|
||||
writeAnyEscapedString<'\''>(str, str + size, buf);
|
||||
}
|
||||
|
||||
|
||||
inline void writeEscapedString(const String & s, WriteBuffer & buf)
|
||||
{
|
||||
writeEscapedString(s.data(), s.size(), buf);
|
||||
}
|
||||
|
||||
|
||||
@ -470,6 +476,10 @@ inline void writeText(const Float32 & x, WriteBuffer & buf) { writeFloatText(x,
|
||||
inline void writeText(const Float64 & x, WriteBuffer & buf) { writeFloatText(x, buf); }
|
||||
inline void writeText(const String & x, WriteBuffer & buf) { writeEscapedString(x, buf); }
|
||||
inline void writeText(const bool & x, WriteBuffer & buf) { writeBoolText(x, buf); }
|
||||
/// в отличие от метода для std::string
|
||||
/// здесь предполагается, что x null-terminated строка.
|
||||
inline void writeText(const char * x, WriteBuffer & buf) { writeEscapedString(x, strlen(x), buf); }
|
||||
inline void writeText(const char * x, size_t size, WriteBuffer & buf) { writeEscapedString(x, size, buf); }
|
||||
|
||||
inline void writeText(const VisitID_t & x, WriteBuffer & buf) { writeIntText(static_cast<const UInt64 &>(x), buf); }
|
||||
inline void writeText(const mysqlxx::Date & x, WriteBuffer & buf) { writeDateText(x, buf); }
|
||||
|
@ -40,6 +40,7 @@ class ProcessListElement;
|
||||
class Macros;
|
||||
class Progress;
|
||||
class Clusters;
|
||||
class QueryLog;
|
||||
|
||||
|
||||
/// имя таблицы -> таблица
|
||||
@ -64,12 +65,30 @@ typedef std::vector<DatabaseAndTableName> Dependencies;
|
||||
*/
|
||||
class Context
|
||||
{
|
||||
public:
|
||||
enum class Interface
|
||||
{
|
||||
TCP = 1,
|
||||
HTTP = 2,
|
||||
OLAP_HTTP = 3,
|
||||
};
|
||||
|
||||
enum class HTTPMethod
|
||||
{
|
||||
UNKNOWN = 0,
|
||||
GET = 1,
|
||||
POST = 2,
|
||||
};
|
||||
|
||||
private:
|
||||
typedef std::shared_ptr<ContextShared> Shared;
|
||||
Shared shared;
|
||||
|
||||
String user; /// Текущий пользователь.
|
||||
Poco::Net::IPAddress ip_address; /// IP-адрес, с которого задан запрос.
|
||||
Interface interface = Interface::TCP;
|
||||
HTTPMethod http_method = HTTPMethod::UNKNOWN; /// NOTE Возможно, перенести это в отдельный struct ClientInfo.
|
||||
|
||||
std::shared_ptr<QuotaForIntervals> quota; /// Текущая квота. По-умолчанию - пустая квота, которая ничего не ограничивает.
|
||||
String current_database; /// Текущая БД.
|
||||
String current_query_id; /// Id текущего запроса.
|
||||
@ -108,6 +127,12 @@ public:
|
||||
String getUser() const { return user; }
|
||||
Poco::Net::IPAddress getIPAddress() const { return ip_address; }
|
||||
|
||||
Interface getInterface() const { return interface; }
|
||||
void setInterface(Interface interface_) { interface = interface_; }
|
||||
|
||||
HTTPMethod getHTTPMethod() const { return http_method; }
|
||||
void setHTTPMethod(HTTPMethod http_method_) { http_method = http_method_; }
|
||||
|
||||
void setQuota(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & address);
|
||||
QuotaForIntervals & getQuota();
|
||||
|
||||
@ -242,6 +267,8 @@ public:
|
||||
|
||||
Compiler & getCompiler();
|
||||
|
||||
QueryLog & getQueryLog();
|
||||
|
||||
/// Позволяет выбрать метод сжатия по условиям, описанным в конфигурационном файле.
|
||||
CompressionMethod chooseCompressionMethod(size_t part_size, double part_size_ratio) const;
|
||||
|
||||
|
23
dbms/include/DB/Interpreters/IInterpreter.h
Normal file
23
dbms/include/DB/Interpreters/IInterpreter.h
Normal file
@ -0,0 +1,23 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/DataStreams/BlockIO.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Интерфейс интерпретаторов разных запросов.
|
||||
*/
|
||||
class IInterpreter
|
||||
{
|
||||
public:
|
||||
/** Для запросов, возвращающих результат (SELECT и похожие), устанавливает в BlockIO поток, из которого можно будет читать этот результат.
|
||||
* Для запросов, принимающих данные (INSERT), устанавливает в BlockIO поток, куда можно писать данные.
|
||||
* Для запросов, которые не требуют данные и ничего не возвращают, BlockIO будет пустым.
|
||||
*/
|
||||
virtual BlockIO execute() = 0;
|
||||
|
||||
virtual ~IInterpreter() {}
|
||||
};
|
||||
|
||||
}
|
@ -3,6 +3,7 @@
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Storages/AlterCommands.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Interpreters/IInterpreter.h>
|
||||
#include <DB/Parsers/ASTIdentifier.h>
|
||||
|
||||
namespace DB
|
||||
@ -13,12 +14,12 @@ namespace DB
|
||||
/** Позволяет добавить или удалить столбец в таблице.
|
||||
* Также позволяет осуществить манипуляции с партициями таблиц семейства MergeTree.
|
||||
*/
|
||||
class InterpreterAlterQuery
|
||||
class InterpreterAlterQuery : public IInterpreter
|
||||
{
|
||||
public:
|
||||
InterpreterAlterQuery(ASTPtr query_ptr_, Context & context_);
|
||||
|
||||
void execute();
|
||||
BlockIO execute() override;
|
||||
|
||||
/** Изменяет список столбцов в метаданных таблицы на диске. Нужно вызывать под TableStructureLock соответствующей таблицы.
|
||||
*/
|
||||
|
@ -1,22 +1,22 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Interpreters/IInterpreter.h>
|
||||
#include <DB/Parsers/ASTIdentifier.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class InterpreterCheckQuery
|
||||
class InterpreterCheckQuery : public IInterpreter
|
||||
{
|
||||
public:
|
||||
InterpreterCheckQuery(ASTPtr query_ptr_, Context & context_);
|
||||
BlockInputStreamPtr execute();
|
||||
DB::Block getSampleBlock();
|
||||
BlockIO execute() override;
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
Context context;
|
||||
DB::Block result;
|
||||
Block result;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Interpreters/IInterpreter.h>
|
||||
#include <DB/Storages/ColumnDefault.h>
|
||||
|
||||
|
||||
@ -11,7 +12,7 @@ namespace DB
|
||||
|
||||
/** Позволяет создать новую таблицу, или создать объект уже существующей таблицы, или создать БД, или создать объект уже существующей БД
|
||||
*/
|
||||
class InterpreterCreateQuery
|
||||
class InterpreterCreateQuery : public IInterpreter
|
||||
{
|
||||
public:
|
||||
InterpreterCreateQuery(ASTPtr query_ptr_, Context & context_);
|
||||
@ -21,7 +22,19 @@ public:
|
||||
* assume_metadata_exists - не проверять наличие файла с метаданными и не создавать его
|
||||
* (для случая выполнения запроса из существующего файла с метаданными).
|
||||
*/
|
||||
StoragePtr execute(bool assume_metadata_exists = false);
|
||||
BlockIO execute() override
|
||||
{
|
||||
executeImpl(false);
|
||||
return {};
|
||||
}
|
||||
|
||||
/** assume_metadata_exists - не проверять наличие файла с метаданными и не создавать его
|
||||
* (для случая выполнения запроса из существующего файла с метаданными).
|
||||
*/
|
||||
void executeLoadExisting()
|
||||
{
|
||||
executeImpl(true);
|
||||
}
|
||||
|
||||
/// Список столбцов с типами в AST.
|
||||
static ASTPtr formatColumns(const NamesAndTypesList & columns);
|
||||
@ -32,6 +45,8 @@ public:
|
||||
const ColumnDefaults & column_defaults);
|
||||
|
||||
private:
|
||||
void executeImpl(bool assume_metadata_exists);
|
||||
|
||||
/// AST в список столбцов с типами. Столбцы типа Nested развернуты в список настоящих столбцов.
|
||||
using ColumnsAndDefaults = std::pair<NamesAndTypesList, ColumnDefaults>;
|
||||
ColumnsAndDefaults parseColumns(ASTPtr expression_list);
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <DB/Parsers/TablePropertiesQueriesASTs.h>
|
||||
#include <DB/Parsers/ASTIdentifier.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Interpreters/IInterpreter.h>
|
||||
#include <DB/DataStreams/OneBlockInputStream.h>
|
||||
#include <DB/DataStreams/BlockIO.h>
|
||||
#include <DB/DataStreams/copyData.h>
|
||||
@ -20,13 +21,13 @@ namespace DB
|
||||
|
||||
/** Вернуть названия и типы столбцов указанной таблицы.
|
||||
*/
|
||||
class InterpreterDescribeQuery
|
||||
class InterpreterDescribeQuery : public IInterpreter
|
||||
{
|
||||
public:
|
||||
InterpreterDescribeQuery(ASTPtr query_ptr_, Context & context_)
|
||||
: query_ptr(query_ptr_), context(context_) {}
|
||||
|
||||
BlockIO execute()
|
||||
BlockIO execute() override
|
||||
{
|
||||
BlockIO res;
|
||||
res.in = executeImpl();
|
||||
@ -35,20 +36,6 @@ public:
|
||||
return res;
|
||||
}
|
||||
|
||||
BlockInputStreamPtr executeAndFormat(WriteBuffer & buf)
|
||||
{
|
||||
Block sample = getSampleBlock();
|
||||
ASTPtr format_ast = typeid_cast<ASTDescribeQuery &>(*query_ptr).format;
|
||||
String format_name = format_ast ? typeid_cast<ASTIdentifier &>(*format_ast).name : context.getDefaultFormat();
|
||||
|
||||
BlockInputStreamPtr in = executeImpl();
|
||||
BlockOutputStreamPtr out = context.getFormatFactory().getOutput(format_name, buf, sample);
|
||||
|
||||
copyData(*in, *out);
|
||||
|
||||
return in;
|
||||
}
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
Context context;
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Interpreters/IInterpreter.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -10,13 +11,13 @@ namespace DB
|
||||
|
||||
/** Позволяет удалить таблицу вместе со всеми данными (DROP), или удалить информацию о таблице из сервера (DETACH).
|
||||
*/
|
||||
class InterpreterDropQuery
|
||||
class InterpreterDropQuery : public IInterpreter
|
||||
{
|
||||
public:
|
||||
InterpreterDropQuery(ASTPtr query_ptr_, Context & context_);
|
||||
|
||||
/// Удаляет таблицу.
|
||||
void execute();
|
||||
BlockIO execute() override;
|
||||
|
||||
/// Удаляет таблицу, уже отцепленную от контекста (Context::detach).
|
||||
static void dropDetachedTable(String database_name, StoragePtr table, Context & context);
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <DB/Parsers/TablePropertiesQueriesASTs.h>
|
||||
#include <DB/Parsers/ASTIdentifier.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Interpreters/IInterpreter.h>
|
||||
#include <DB/DataStreams/OneBlockInputStream.h>
|
||||
#include <DB/DataStreams/BlockIO.h>
|
||||
#include <DB/DataStreams/FormatFactory.h>
|
||||
@ -17,13 +18,13 @@ namespace DB
|
||||
|
||||
/** Проверить, существует ли таблица. Вернуть одну строку с одним столбцом result типа UInt8 со значением 0 или 1.
|
||||
*/
|
||||
class InterpreterExistsQuery
|
||||
class InterpreterExistsQuery : public IInterpreter
|
||||
{
|
||||
public:
|
||||
InterpreterExistsQuery(ASTPtr query_ptr_, Context & context_)
|
||||
: query_ptr(query_ptr_), context(context_) {}
|
||||
|
||||
BlockIO execute()
|
||||
BlockIO execute() override
|
||||
{
|
||||
BlockIO res;
|
||||
res.in = executeImpl();
|
||||
@ -32,20 +33,6 @@ public:
|
||||
return res;
|
||||
}
|
||||
|
||||
BlockInputStreamPtr executeAndFormat(WriteBuffer & buf)
|
||||
{
|
||||
Block sample = getSampleBlock();
|
||||
ASTPtr format_ast = typeid_cast<ASTExistsQuery &>(*query_ptr).format;
|
||||
String format_name = format_ast ? typeid_cast<ASTIdentifier &>(*format_ast).name : context.getDefaultFormat();
|
||||
|
||||
BlockInputStreamPtr in = executeImpl();
|
||||
BlockOutputStreamPtr out = context.getFormatFactory().getOutput(format_name, buf, sample);
|
||||
|
||||
copyData(*in, *out);
|
||||
|
||||
return in;
|
||||
}
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
Context context;
|
||||
|
19
dbms/include/DB/Interpreters/InterpreterFactory.h
Normal file
19
dbms/include/DB/Interpreters/InterpreterFactory.h
Normal file
@ -0,0 +1,19 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Interpreters/IInterpreter.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class InterpreterFactory
|
||||
{
|
||||
public:
|
||||
static SharedPtr<IInterpreter> get(
|
||||
ASTPtr & query,
|
||||
Context & context,
|
||||
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete);
|
||||
};
|
||||
|
||||
}
|
@ -3,6 +3,7 @@
|
||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||
#include <DB/DataStreams/BlockIO.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Interpreters/IInterpreter.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -11,23 +12,17 @@ namespace DB
|
||||
|
||||
/** Интерпретирует запрос INSERT.
|
||||
*/
|
||||
class InterpreterInsertQuery
|
||||
class InterpreterInsertQuery : public IInterpreter
|
||||
{
|
||||
public:
|
||||
InterpreterInsertQuery(ASTPtr query_ptr_, Context & context_);
|
||||
|
||||
/** Выполнить запрос.
|
||||
* remaining_data_istr, если не nullptr, может содержать нераспарсенные данные для вставки.
|
||||
* (заранее может быть считан в оперативку для парсинга лишь небольшой кусок запроса, который содержит не все данные)
|
||||
*/
|
||||
void execute(ReadBuffer * remaining_data_istr);
|
||||
|
||||
/** Подготовить запрос к выполнению. Вернуть потоки блоков
|
||||
* - поток, в который можно писать данные для выполнения запроса, если INSERT;
|
||||
* - поток, из которого можно читать результат выполнения запроса, если SELECT и подобные;
|
||||
* Или ничего, если запрос INSERT SELECT (самодостаточный запрос - не принимает входные данные, не отдаёт результат).
|
||||
*/
|
||||
BlockIO execute();
|
||||
BlockIO execute() override;
|
||||
|
||||
private:
|
||||
StoragePtr getTable();
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Parsers/ASTOptimizeQuery.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Interpreters/IInterpreter.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -11,7 +12,7 @@ namespace DB
|
||||
|
||||
/** Просто вызвать метод optimize у таблицы.
|
||||
*/
|
||||
class InterpreterOptimizeQuery
|
||||
class InterpreterOptimizeQuery : public IInterpreter
|
||||
{
|
||||
public:
|
||||
InterpreterOptimizeQuery(ASTPtr query_ptr_, Context & context_)
|
||||
@ -19,12 +20,13 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
void execute()
|
||||
BlockIO execute() override
|
||||
{
|
||||
const ASTOptimizeQuery & ast = typeid_cast<const ASTOptimizeQuery &>(*query_ptr);
|
||||
StoragePtr table = context.getTable(ast.database, ast.table);
|
||||
auto table_lock = table->lockStructure(true);
|
||||
table->optimize(context.getSettings());
|
||||
return {};
|
||||
}
|
||||
|
||||
private:
|
||||
|
@ -1,50 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Core/QueryProcessingStage.h>
|
||||
#include <DB/DataStreams/BlockIO.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
/** Интерпретирует произвольный запрос.
|
||||
*/
|
||||
class InterpreterQuery
|
||||
{
|
||||
public:
|
||||
InterpreterQuery(ASTPtr query_ptr_, Context & context_, QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
|
||||
|
||||
/** Выполнить запрос.
|
||||
*
|
||||
* ostr - куда писать результат выполнения запроса, если он есть.
|
||||
*
|
||||
* remaining_data_istr, если не nullptr, может содержать нераспарсенный остаток запроса с данными.
|
||||
* (заранее может быть считан в оперативку для парсинга лишь небольшой кусок запроса, который содержит не все данные)
|
||||
*
|
||||
* В query_plan,
|
||||
* после выполнения запроса, может быть записан BlockInputStreamPtr,
|
||||
* использовавшийся при выполнении запроса,
|
||||
* чтобы можно было получить информацию о том, как выполнялся запрос.
|
||||
*/
|
||||
void execute(WriteBuffer & ostr, ReadBuffer * remaining_data_istr, BlockInputStreamPtr & query_plan);
|
||||
|
||||
/** Подготовить запрос к выполнению. Вернуть потоки блоков, используя которые можно выполнить запрос.
|
||||
*/
|
||||
BlockIO execute();
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
Context context;
|
||||
QueryProcessingStage::Enum stage;
|
||||
|
||||
void throwIfReadOnly()
|
||||
{
|
||||
if (context.getSettingsRef().limits.readonly)
|
||||
throw Exception("Cannot execute query in readonly mode", ErrorCodes::READONLY);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
}
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Interpreters/IInterpreter.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -10,11 +11,11 @@ namespace DB
|
||||
|
||||
/** Переименовать одну или несколько таблиц.
|
||||
*/
|
||||
class InterpreterRenameQuery
|
||||
class InterpreterRenameQuery : public IInterpreter
|
||||
{
|
||||
public:
|
||||
InterpreterRenameQuery(ASTPtr query_ptr_, Context & context_);
|
||||
void execute();
|
||||
BlockIO execute() override;
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <DB/Core/QueryProcessingStage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Interpreters/IInterpreter.h>
|
||||
#include <DB/Interpreters/ExpressionActions.h>
|
||||
#include <DB/DataStreams/IBlockInputStream.h>
|
||||
|
||||
@ -15,7 +16,7 @@ class SubqueryForSet;
|
||||
|
||||
/** Интерпретирует запрос SELECT. Возвращает поток блоков с результатами выполнения запроса до стадии to_stage.
|
||||
*/
|
||||
class InterpreterSelectQuery
|
||||
class InterpreterSelectQuery : public IInterpreter
|
||||
{
|
||||
public:
|
||||
/** to_stage
|
||||
@ -66,17 +67,12 @@ public:
|
||||
/** Выполнить запрос, возможно являющиийся цепочкой UNION ALL.
|
||||
* Получить поток блоков для чтения
|
||||
*/
|
||||
BlockInputStreamPtr execute();
|
||||
BlockIO execute() override;
|
||||
|
||||
/** Выполнить запрос без объединения потоков, если это возможно.
|
||||
*/
|
||||
const BlockInputStreams & executeWithoutUnion();
|
||||
|
||||
/** Выполнить запрос, записать результат в нужном формате в buf.
|
||||
* BlockInputStreamPtr возвращается, чтобы можно было потом получить информацию о плане выполнения запроса.
|
||||
*/
|
||||
BlockInputStreamPtr executeAndFormat(WriteBuffer & buf);
|
||||
|
||||
DataTypes getReturnTypes();
|
||||
Block getSampleBlock();
|
||||
|
||||
@ -100,7 +96,7 @@ private:
|
||||
// Переименовать столбцы каждого запроса цепочки UNION ALL в такие же имена, как в первом запросе.
|
||||
void renameColumns();
|
||||
|
||||
/** Из какой таблицы читать. JOIN-ы не поддерживаются.
|
||||
/** Из какой таблицы читать. При JOIN, возвращается "левая" таблицы.
|
||||
*/
|
||||
void getDatabaseAndTableNames(String & database_name, String & table_name);
|
||||
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <DB/Parsers/ASTSetQuery.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Interpreters/IInterpreter.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -10,20 +11,20 @@ namespace DB
|
||||
|
||||
/** Установить один или несколько параметров, для сессии или глобально... или для текущего запроса.
|
||||
*/
|
||||
class InterpreterSetQuery
|
||||
class InterpreterSetQuery : public IInterpreter
|
||||
{
|
||||
public:
|
||||
InterpreterSetQuery(ASTPtr query_ptr_, Context & context_)
|
||||
: query_ptr(query_ptr_), context(context_) {}
|
||||
|
||||
|
||||
/** Обычный запрос SET. Задать настройку на сессию или глобальную (если указано GLOBAL).
|
||||
*/
|
||||
void execute()
|
||||
BlockIO execute() override
|
||||
{
|
||||
ASTSetQuery & ast = typeid_cast<ASTSetQuery &>(*query_ptr);
|
||||
Context & target = ast.global ? context.getGlobalContext() : context.getSessionContext();
|
||||
executeImpl(ast, target);
|
||||
return {};
|
||||
}
|
||||
|
||||
/** Задать настроку для текущего контекста (контекста запроса).
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <DB/Parsers/ASTIdentifier.h>
|
||||
#include <DB/Parsers/formatAST.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Interpreters/IInterpreter.h>
|
||||
#include <DB/DataStreams/OneBlockInputStream.h>
|
||||
#include <DB/DataStreams/BlockIO.h>
|
||||
#include <DB/DataStreams/copyData.h>
|
||||
@ -18,13 +19,13 @@ namespace DB
|
||||
|
||||
/** Вернуть одну строку с одним столбцом statement типа String с текстом запроса, создающего указанную таблицу.
|
||||
*/
|
||||
class InterpreterShowCreateQuery
|
||||
class InterpreterShowCreateQuery : public IInterpreter
|
||||
{
|
||||
public:
|
||||
InterpreterShowCreateQuery(ASTPtr query_ptr_, Context & context_)
|
||||
: query_ptr(query_ptr_), context(context_) {}
|
||||
|
||||
BlockIO execute()
|
||||
BlockIO execute() override
|
||||
{
|
||||
BlockIO res;
|
||||
res.in = executeImpl();
|
||||
@ -33,20 +34,6 @@ public:
|
||||
return res;
|
||||
}
|
||||
|
||||
BlockInputStreamPtr executeAndFormat(WriteBuffer & buf)
|
||||
{
|
||||
Block sample = getSampleBlock();
|
||||
ASTPtr format_ast = typeid_cast<ASTShowCreateQuery &>(*query_ptr).format;
|
||||
String format_name = format_ast ? typeid_cast<ASTIdentifier &>(*format_ast).name : context.getDefaultFormat();
|
||||
|
||||
BlockInputStreamPtr in = executeImpl();
|
||||
BlockOutputStreamPtr out = context.getFormatFactory().getOutput(format_name, buf, sample);
|
||||
|
||||
copyData(*in, *out);
|
||||
|
||||
return in;
|
||||
}
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
Context context;
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <DB/IO/ReadBufferFromString.h>
|
||||
|
||||
#include <DB/Interpreters/executeQuery.h>
|
||||
#include <DB/Interpreters/IInterpreter.h>
|
||||
|
||||
#include <DB/Parsers/ASTQueryWithOutput.h>
|
||||
#include <DB/Parsers/ASTIdentifier.h>
|
||||
@ -14,26 +15,17 @@ namespace DB
|
||||
|
||||
/** Вернуть список запросов, исполняющихся прямо сейчас.
|
||||
*/
|
||||
class InterpreterShowProcesslistQuery
|
||||
class InterpreterShowProcesslistQuery : public IInterpreter
|
||||
{
|
||||
public:
|
||||
InterpreterShowProcesslistQuery(ASTPtr query_ptr_, Context & context_)
|
||||
: query_ptr(query_ptr_), context(context_) {}
|
||||
|
||||
BlockIO execute()
|
||||
BlockIO execute() override
|
||||
{
|
||||
return executeQuery(getRewrittenQuery(), context, true);
|
||||
}
|
||||
|
||||
BlockInputStreamPtr executeAndFormat(WriteBuffer & buf)
|
||||
{
|
||||
String query = getRewrittenQuery();
|
||||
ReadBufferFromString in(query);
|
||||
BlockInputStreamPtr query_plan;
|
||||
executeQuery(in, buf, context, query_plan, true);
|
||||
return query_plan;
|
||||
}
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
Context context;
|
||||
|
@ -1,8 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/DataStreams/BlockIO.h>
|
||||
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Interpreters/IInterpreter.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -12,13 +11,12 @@ namespace DB
|
||||
/** Вывести список имён таблиц/баз данных по некоторым условиям.
|
||||
* Интерпретирует запрос путём замены его на запрос SELECT из таблицы system.tables или system.databases.
|
||||
*/
|
||||
class InterpreterShowTablesQuery
|
||||
class InterpreterShowTablesQuery : public IInterpreter
|
||||
{
|
||||
public:
|
||||
InterpreterShowTablesQuery(ASTPtr query_ptr_, Context & context_);
|
||||
|
||||
BlockIO execute();
|
||||
BlockInputStreamPtr executeAndFormat(WriteBuffer & buf);
|
||||
BlockIO execute() override;
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <DB/Parsers/ASTUseQuery.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Interpreters/IInterpreter.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -10,16 +11,17 @@ namespace DB
|
||||
|
||||
/** Выбрать БД по-умолчанию для сессии.
|
||||
*/
|
||||
class InterpreterUseQuery
|
||||
class InterpreterUseQuery : public IInterpreter
|
||||
{
|
||||
public:
|
||||
InterpreterUseQuery(ASTPtr query_ptr_, Context & context_)
|
||||
: query_ptr(query_ptr_), context(context_) {}
|
||||
|
||||
void execute()
|
||||
BlockIO execute() override
|
||||
{
|
||||
const String & new_database = typeid_cast<const ASTUseQuery &>(*query_ptr).database;
|
||||
context.getSessionContext().setCurrentDatabase(new_database);
|
||||
return {};
|
||||
}
|
||||
|
||||
private:
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <map>
|
||||
#include <list>
|
||||
#include <Poco/SharedPtr.h>
|
||||
#include <memory>
|
||||
#include <Poco/Mutex.h>
|
||||
#include <Poco/Condition.h>
|
||||
#include <Poco/Net/IPAddress.h>
|
||||
@ -13,6 +13,7 @@
|
||||
#include <DB/Core/ErrorCodes.h>
|
||||
#include <DB/Common/MemoryTracker.h>
|
||||
#include <DB/IO/WriteHelpers.h>
|
||||
#include <DB/Interpreters/QueryPriorities.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -36,13 +37,16 @@ struct ProcessListElement
|
||||
|
||||
MemoryTracker memory_tracker;
|
||||
|
||||
QueryPriorities::Handle priority_handle;
|
||||
|
||||
bool is_cancelled = false;
|
||||
|
||||
|
||||
ProcessListElement(const String & query_, const String & user_,
|
||||
const String & query_id_, const Poco::Net::IPAddress & ip_address_,
|
||||
size_t max_memory_usage)
|
||||
: query(query_), user(user_), query_id(query_id_), ip_address(ip_address_), memory_tracker(max_memory_usage)
|
||||
size_t max_memory_usage, QueryPriorities::Handle && priority_handle_)
|
||||
: query(query_), user(user_), query_id(query_id_), ip_address(ip_address_), memory_tracker(max_memory_usage),
|
||||
priority_handle(std::move(priority_handle_))
|
||||
{
|
||||
current_memory_tracker = &memory_tracker;
|
||||
}
|
||||
@ -55,126 +59,81 @@ struct ProcessListElement
|
||||
bool update(const Progress & value)
|
||||
{
|
||||
progress.incrementPiecewiseAtomically(value);
|
||||
|
||||
if (priority_handle)
|
||||
priority_handle->waitIfNeed(std::chrono::seconds(1)); /// NOTE Можно сделать настраиваемым таймаут.
|
||||
|
||||
return !is_cancelled;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class ProcessList;
|
||||
|
||||
|
||||
/// Держит итератор на список, и удаляет элемент из списка в деструкторе.
|
||||
class ProcessListEntry
|
||||
{
|
||||
private:
|
||||
using Container = std::list<ProcessListElement>;
|
||||
|
||||
ProcessList & parent;
|
||||
Container::iterator it;
|
||||
public:
|
||||
ProcessListEntry(ProcessList & parent_, Container::iterator it_)
|
||||
: parent(parent_), it(it_) {}
|
||||
|
||||
~ProcessListEntry();
|
||||
|
||||
ProcessListElement * operator->() { return &*it; }
|
||||
const ProcessListElement * operator->() const { return &*it; }
|
||||
|
||||
ProcessListElement & get() { return *it; }
|
||||
const ProcessListElement & get() const { return *it; }
|
||||
};
|
||||
|
||||
|
||||
class ProcessList
|
||||
{
|
||||
friend class Entry;
|
||||
friend class ProcessListEntry;
|
||||
public:
|
||||
using Element = ProcessListElement;
|
||||
using Entry = ProcessListEntry;
|
||||
|
||||
/// list, чтобы итераторы не инвалидировались. NOTE: можно заменить на cyclic buffer, но почти незачем.
|
||||
typedef std::list<Element> Containter;
|
||||
using Container = std::list<Element>;
|
||||
/// Query_id -> Element *
|
||||
typedef std::unordered_map<String, Element *> QueryToElement;
|
||||
using QueryToElement = std::unordered_map<String, Element *>;
|
||||
/// User -> Query_id -> Element *
|
||||
typedef std::unordered_map<String, QueryToElement> UserToQueries;
|
||||
using UserToQueries = std::unordered_map<String, QueryToElement>;
|
||||
|
||||
private:
|
||||
mutable Poco::FastMutex mutex;
|
||||
mutable Poco::Condition have_space; /// Количество одновременно выполняющихся запросов стало меньше максимального.
|
||||
|
||||
Containter cont;
|
||||
Container cont;
|
||||
size_t cur_size; /// В C++03 std::list::size не O(1).
|
||||
size_t max_size; /// Если 0 - не ограничено. Иначе, если пытаемся добавить больше - кидается исключение.
|
||||
UserToQueries user_to_queries;
|
||||
|
||||
/// Держит итератор на список, и удаляет элемент из списка в деструкторе.
|
||||
class Entry
|
||||
{
|
||||
private:
|
||||
ProcessList & parent;
|
||||
Containter::iterator it;
|
||||
public:
|
||||
Entry(ProcessList & parent_, Containter::iterator it_)
|
||||
: parent(parent_), it(it_) {}
|
||||
|
||||
~Entry()
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(parent.mutex);
|
||||
|
||||
/// В случае, если запрос отменяется, данные о нем удаляются из мапа в момент отмены.
|
||||
if (!it->is_cancelled && !it->query_id.empty())
|
||||
{
|
||||
UserToQueries::iterator queries = parent.user_to_queries.find(it->user);
|
||||
if (queries != parent.user_to_queries.end())
|
||||
{
|
||||
QueryToElement::iterator element = queries->second.find(it->query_id);
|
||||
if (element != queries->second.end())
|
||||
queries->second.erase(element);
|
||||
}
|
||||
}
|
||||
|
||||
parent.cont.erase(it);
|
||||
--parent.cur_size;
|
||||
parent.have_space.signal();
|
||||
}
|
||||
|
||||
Element * operator->() { return &*it; }
|
||||
const Element * operator->() const { return &*it; }
|
||||
|
||||
Element & get() { return *it; }
|
||||
const Element & get() const { return *it; }
|
||||
};
|
||||
QueryPriorities priorities;
|
||||
|
||||
public:
|
||||
ProcessList(size_t max_size_ = 0) : cur_size(0), max_size(max_size_) {}
|
||||
|
||||
typedef Poco::SharedPtr<Entry> EntryPtr;
|
||||
typedef std::shared_ptr<ProcessListEntry> EntryPtr;
|
||||
|
||||
/** Зарегистрировать выполняющийся запрос. Возвращает refcounted объект, который удаляет запрос из списка при уничтожении.
|
||||
* Если выполняющихся запросов сейчас слишком много - ждать не более указанного времени.
|
||||
* Если времени не хватило - кинуть исключение.
|
||||
*/
|
||||
EntryPtr insert(const String & query_, const String & user_, const String & query_id_, const Poco::Net::IPAddress & ip_address_,
|
||||
size_t max_memory_usage = 0, size_t max_wait_milliseconds = DEFAULT_QUERIES_QUEUE_WAIT_TIME_MS, bool replace_running_query = false)
|
||||
{
|
||||
EntryPtr res;
|
||||
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
|
||||
if (max_size && cur_size >= max_size && (!max_wait_milliseconds || !have_space.tryWait(mutex, max_wait_milliseconds)))
|
||||
throw Exception("Too much simultaneous queries. Maximum: " + toString(max_size), ErrorCodes::TOO_MUCH_SIMULTANEOUS_QUERIES);
|
||||
|
||||
if (!query_id_.empty())
|
||||
{
|
||||
UserToQueries::iterator queries = user_to_queries.find(user_);
|
||||
|
||||
if (queries != user_to_queries.end())
|
||||
{
|
||||
QueryToElement::iterator element = queries->second.find(query_id_);
|
||||
if (element != queries->second.end())
|
||||
{
|
||||
if (!replace_running_query)
|
||||
throw Exception("Query with id = " + query_id_ + " is already running.",
|
||||
ErrorCodes::QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING);
|
||||
element->second->is_cancelled = true;
|
||||
/// В случае если запрос отменяется, данные о нем удаляются из мапа в момент отмены.
|
||||
queries->second.erase(element);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
++cur_size;
|
||||
|
||||
res = new Entry(*this, cont.emplace(cont.end(), query_, user_, query_id_, ip_address_, max_memory_usage));
|
||||
|
||||
if (!query_id_.empty())
|
||||
user_to_queries[user_][query_id_] = &res->get();
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
size_t max_memory_usage, size_t max_wait_milliseconds, bool replace_running_query, QueryPriorities::Priority priority);
|
||||
|
||||
/// Количество одновременно выполняющихся запросов.
|
||||
size_t size() const { return cur_size; }
|
||||
|
||||
/// Получить текущее состояние (копию) списка запросов.
|
||||
Containter get() const
|
||||
Container get() const
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
return cont;
|
||||
|
127
dbms/include/DB/Interpreters/QueryLog.h
Normal file
127
dbms/include/DB/Interpreters/QueryLog.h
Normal file
@ -0,0 +1,127 @@
|
||||
#pragma once
|
||||
|
||||
#include <thread>
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <Poco/Net/IPAddress.h>
|
||||
#include <DB/Core/Types.h>
|
||||
#include <DB/Common/ConcurrentBoundedQueue.h>
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
/** Позволяет логгировать информацию о выполнении запросов:
|
||||
* - о начале выполнения запроса;
|
||||
* - метрики производительности, после выполнения запроса;
|
||||
* - об ошибках при выполнении запроса.
|
||||
*
|
||||
* Логгирование производится асинхронно. Данные передаются в очередь, откуда их читает отдельный поток.
|
||||
* Этот поток записывает лог в предназначенную для этого таблицу не чаще, чем с заданной периодичностью.
|
||||
*/
|
||||
|
||||
/** Что логгировать.
|
||||
* Структура может меняться при изменении версии сервера.
|
||||
* Если при первой записи обнаруживается, что имеющаяся таблица с логами имеет неподходящую стрктуру,
|
||||
* то эта таблица переименовывается (откладывается в сторону) и создаётся новая таблица.
|
||||
*/
|
||||
struct QueryLogElement
|
||||
{
|
||||
enum Type
|
||||
{
|
||||
SHUTDOWN = 0, /// Эта запись имеет служебное значение.
|
||||
QUERY_START = 1,
|
||||
QUERY_FINISH = 2,
|
||||
EXCEPTION_BEFORE_START = 3,
|
||||
EXCEPTION_WHILE_PROCESSING = 4,
|
||||
};
|
||||
|
||||
Type type = QUERY_START;
|
||||
|
||||
/// В зависимости от типа, не все поля могут быть заполнены.
|
||||
|
||||
time_t event_time{};
|
||||
time_t query_start_time{};
|
||||
UInt64 query_duration_ms{};
|
||||
|
||||
UInt64 read_rows{};
|
||||
UInt64 read_bytes{};
|
||||
|
||||
UInt64 result_rows{};
|
||||
UInt64 result_bytes{};
|
||||
|
||||
UInt64 memory_usage{};
|
||||
|
||||
String query;
|
||||
|
||||
String exception;
|
||||
String stack_trace;
|
||||
|
||||
Context::Interface interface = Context::Interface::TCP;
|
||||
Context::HTTPMethod http_method = Context::HTTPMethod::UNKNOWN;
|
||||
Poco::Net::IPAddress ip_address;
|
||||
String user;
|
||||
String query_id;
|
||||
};
|
||||
|
||||
|
||||
#define DBMS_QUERY_LOG_QUEUE_SIZE 1024
|
||||
|
||||
class Context;
|
||||
|
||||
|
||||
class QueryLog : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
|
||||
/** Передаётся имя таблицы, в которую писать лог.
|
||||
* Если таблица не существует, то она создаётся с движком MergeTree, с ключём по event_time.
|
||||
* Если таблица существует, то проверяется, подходящая ли у неё структура.
|
||||
* Если структура подходящая, то будет использоваться эта таблица.
|
||||
* Если нет - то существующая таблица переименовывается в такую же, но с добавлением суффикса _N на конце,
|
||||
* где N - минимальное число, начиная с 1 такое, что таблицы с таким именем ещё нет;
|
||||
* и создаётся новая таблица, как будто существующей таблицы не было.
|
||||
*/
|
||||
QueryLog(Context & context_, const String & database_name_, const String & table_name_, size_t flush_interval_milliseconds_);
|
||||
~QueryLog();
|
||||
|
||||
/** Добавить запись в лог.
|
||||
* Сохранение в таблицу делается асинхронно, и в случае сбоя, запись может никуда не попасть.
|
||||
*/
|
||||
void add(const QueryLogElement & element)
|
||||
{
|
||||
/// Здесь может быть блокировка. Возможно, в случае переполнения очереди, лучше сразу кидать эксепшен. Или даже отказаться от логгирования запроса.
|
||||
queue.push(element);
|
||||
}
|
||||
|
||||
private:
|
||||
Context & context;
|
||||
const String database_name;
|
||||
const String table_name;
|
||||
StoragePtr table;
|
||||
const size_t flush_interval_milliseconds;
|
||||
|
||||
/// Очередь всё-таки ограничена. Но размер достаточно большой, чтобы не блокироваться во всех нормальных ситуациях.
|
||||
ConcurrentBoundedQueue<QueryLogElement> queue {DBMS_QUERY_LOG_QUEUE_SIZE};
|
||||
|
||||
/** Данные, которые были вынуты из очереди. Здесь данные накапливаются, пока не пройдёт достаточное количество времени.
|
||||
* Можно было бы использовать двойную буферизацию, но предполагается,
|
||||
* что запись в таблицу с логом будет быстрее, чем обработка большой пачки запросов.
|
||||
*/
|
||||
std::vector<QueryLogElement> data;
|
||||
|
||||
Logger * log {&Logger::get("QueryLog")};
|
||||
|
||||
/** В этом потоке данные вынимаются из queue, складываются в data, а затем вставляются в таблицу.
|
||||
*/
|
||||
std::thread saving_thread;
|
||||
|
||||
void threadFunction();
|
||||
static Block createBlock();
|
||||
void flush();
|
||||
};
|
||||
|
||||
|
||||
}
|
117
dbms/include/DB/Interpreters/QueryPriorities.h
Normal file
117
dbms/include/DB/Interpreters/QueryPriorities.h
Normal file
@ -0,0 +1,117 @@
|
||||
#pragma once
|
||||
|
||||
#include <map>
|
||||
#include <mutex>
|
||||
#include <condition_variable>
|
||||
#include <memory>
|
||||
#include <chrono>
|
||||
|
||||
|
||||
/** Реализует приоритеты запросов.
|
||||
* Позволяет приостанавливать выполнение запроса, если выполняется хотя бы один более приоритетный запрос.
|
||||
*
|
||||
* Величина приоритета - целое число, чем меньше - тем больше приоритет.
|
||||
*
|
||||
* Приоритет 0 считается особенным - запросы с таким приоритетом выполняются всегда,
|
||||
* не зависят от других запросов и не влияют на другие запросы.
|
||||
* То есть 0 означает - не использовать приоритеты.
|
||||
*
|
||||
* NOTE Возможности сделать лучше:
|
||||
* - реализовать ограничение на максимальное количество запросов с таким приоритетом.
|
||||
*/
|
||||
class QueryPriorities
|
||||
{
|
||||
public:
|
||||
using Priority = int;
|
||||
|
||||
private:
|
||||
friend struct Handle;
|
||||
|
||||
using Count = int;
|
||||
|
||||
/// Количество выполняющихся сейчас запросов с заданным приоритетом.
|
||||
using Container = std::map<Priority, Count>;
|
||||
|
||||
std::mutex mutex;
|
||||
std::condition_variable condvar;
|
||||
Container container;
|
||||
|
||||
|
||||
/** Если есть более приоритетные запросы - спать, пока они не перестанут быть или не истечёт таймаут.
|
||||
* Возвращает true, если более приоритетные запросы исчезли на момент возврата из функции, false, если истёк таймаут.
|
||||
*/
|
||||
template <typename Duration>
|
||||
bool waitIfNeed(Priority priority, Duration timeout)
|
||||
{
|
||||
if (0 == priority)
|
||||
return true;
|
||||
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
|
||||
while (true)
|
||||
{
|
||||
/// Если ли хотя бы один более приоритетный запрос?
|
||||
bool found = false;
|
||||
for (const auto & value : container)
|
||||
{
|
||||
if (value.first >= priority)
|
||||
break;
|
||||
|
||||
if (value.second > 0)
|
||||
{
|
||||
found = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (!found)
|
||||
return true;
|
||||
|
||||
if (std::cv_status::timeout == condvar.wait_for(lock, timeout))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
public:
|
||||
struct HandleImpl
|
||||
{
|
||||
private:
|
||||
QueryPriorities & parent;
|
||||
QueryPriorities::Container::value_type & value;
|
||||
|
||||
public:
|
||||
HandleImpl(QueryPriorities & parent_, QueryPriorities::Container::value_type & value_)
|
||||
: parent(parent_), value(value_) {}
|
||||
|
||||
~HandleImpl()
|
||||
{
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(parent.mutex);
|
||||
--value.second;
|
||||
}
|
||||
parent.condvar.notify_all();
|
||||
}
|
||||
|
||||
template <typename Duration>
|
||||
bool waitIfNeed(Duration timeout)
|
||||
{
|
||||
return parent.waitIfNeed(value.first, timeout);
|
||||
}
|
||||
};
|
||||
|
||||
using Handle = std::shared_ptr<HandleImpl>;
|
||||
|
||||
/** Зарегистрировать, что запрос с заданным приоритетом выполняется.
|
||||
* Возвращается объект, в деструкторе которого, запись о запросе удаляется.
|
||||
*/
|
||||
Handle insert(Priority priority)
|
||||
{
|
||||
if (0 == priority)
|
||||
return {};
|
||||
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
auto it = container.emplace(priority, 0).first;
|
||||
++it->second;
|
||||
return std::make_shared<HandleImpl>(*this, *it);
|
||||
}
|
||||
};
|
@ -83,7 +83,7 @@ struct QuotaForInterval
|
||||
|
||||
/// Увеличить соответствующее значение.
|
||||
void addQuery(time_t current_time, const String & quota_name);
|
||||
void addError(time_t current_time, const String & quota_name);
|
||||
void addError(time_t current_time, const String & quota_name) noexcept;
|
||||
|
||||
/// Проверить, не превышена ли квота уже. Если превышена - кидает исключение.
|
||||
void checkExceeded(time_t current_time, const String & quota_name);
|
||||
@ -139,7 +139,7 @@ public:
|
||||
void setMax(const QuotaForIntervals & quota);
|
||||
|
||||
void addQuery(time_t current_time);
|
||||
void addError(time_t current_time);
|
||||
void addError(time_t current_time) noexcept;
|
||||
|
||||
void checkExceeded(time_t current_time);
|
||||
|
||||
|
@ -97,6 +97,14 @@ struct Settings
|
||||
M(SettingUInt64, parallel_replicas_count, 0) \
|
||||
M(SettingUInt64, parallel_replica_offset, 0) \
|
||||
\
|
||||
/** Тихо пропускать недоступные шарды. */ \
|
||||
M(SettingBool, skip_unavailable_shards, false) \
|
||||
\
|
||||
/** Не мерджить состояния агрегации с разных серверов при распределённой обработке запроса \
|
||||
* - на случай, когда доподлинно известно, что на разных шардах разные ключи. \
|
||||
*/ \
|
||||
M(SettingBool, distributed_group_by_no_merge, false) \
|
||||
\
|
||||
/** Тонкие настройки для чтения из MergeTree */ \
|
||||
\
|
||||
/** Если из одного файла читается хотя бы столько строк, чтение можно распараллелить. */ \
|
||||
@ -132,6 +140,12 @@ struct Settings
|
||||
\
|
||||
/** Позволяет выбирать метод сжатия данных при записи */\
|
||||
M(SettingCompressionMethod, network_compression_method, CompressionMethod::LZ4) \
|
||||
\
|
||||
/** Приоритет запроса. 1 - самый высокий, больше - ниже; 0 - не использовать приоритеты. */ \
|
||||
M(SettingUInt64, priority, 0) \
|
||||
\
|
||||
/** Логгировать запросы и писать лог в системную таблицу. */ \
|
||||
M(SettingBool, log_queries, 0) \
|
||||
|
||||
/// Всевозможные ограничения на выполнение запроса.
|
||||
Limits limits;
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Core/QueryProcessingStage.h>
|
||||
#include <DB/Interpreters/InterpreterQuery.h>
|
||||
#include <DB/DataStreams/BlockIO.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
ASTLiteral() = default;
|
||||
ASTLiteral(const StringRange range_, const Field & value_) : ASTWithAlias(range_), value(value_) {}
|
||||
|
||||
String getColumnName() const override { return apply_visitor(FieldVisitorToString(), value); }
|
||||
String getColumnName() const override;
|
||||
|
||||
/** Получить текст, который идентифицирует этот элемент. */
|
||||
String getID() const override { return "Literal_" + apply_visitor(FieldVisitorDump(), value); }
|
||||
|
@ -6,7 +6,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
/** Запрос с секцией FORMAT.
|
||||
*/
|
||||
class ASTQueryWithOutput : public IAST
|
||||
@ -16,6 +15,11 @@ public:
|
||||
|
||||
ASTQueryWithOutput() = default;
|
||||
ASTQueryWithOutput(const StringRange range_) : IAST(range_) {}
|
||||
|
||||
/** Возвращает указатель на формат. Если типом объекта является ASTSelectQuery,
|
||||
* то эта функция возвращает указатель на формат из последнего SELECT'а цепочки UNION ALL.
|
||||
*/
|
||||
virtual const IAST * getFormat() const { return format.get(); }
|
||||
};
|
||||
|
||||
|
||||
|
@ -34,6 +34,9 @@ public:
|
||||
|
||||
ASTPtr clone() const override;
|
||||
|
||||
/// Возвращает указатель на формат из последнего SELECT'а цепочки UNION ALL.
|
||||
const IAST * getFormat() const override;
|
||||
|
||||
public:
|
||||
bool distinct = false;
|
||||
ASTPtr select_expression_list;
|
||||
|
@ -100,6 +100,20 @@ protected:
|
||||
};
|
||||
|
||||
|
||||
/** Массив литералов.
|
||||
* Массивы могут распарситься и как применение оператора [].
|
||||
* Но парсинг всего массива как целой константы серьёзно ускоряет анализ выражений в случае очень больших массивов.
|
||||
* Мы пробуем распарсить массив как массив литералов сначала (fast path),
|
||||
* а если не получилось (когда массив состоит из сложных выражений) - парсим как применение оператора [] (slow path).
|
||||
*/
|
||||
class ParserArrayOfLiterals : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const { return "array"; }
|
||||
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
|
||||
};
|
||||
|
||||
|
||||
/** Литерал - одно из: NULL, UInt64, Int64, Float64, String.
|
||||
*/
|
||||
class ParserLiteral : public IParserBase
|
||||
|
@ -42,9 +42,6 @@ public:
|
||||
if (right != rhs.right)
|
||||
return right < rhs.right;
|
||||
|
||||
if (level != rhs.level)
|
||||
return level < rhs.level;
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
@ -53,7 +50,6 @@ public:
|
||||
{
|
||||
return left_month == rhs.left_month /// Куски за разные месяцы не объединяются
|
||||
&& right_month == rhs.right_month
|
||||
&& level > rhs.level
|
||||
&& left_date <= rhs.left_date
|
||||
&& right_date >= rhs.right_date
|
||||
&& left <= rhs.left
|
||||
|
@ -15,115 +15,13 @@ namespace DB
|
||||
{
|
||||
|
||||
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wsign-compare"
|
||||
|
||||
/** Более точное сравнение.
|
||||
* Отличается от Field::operator< и Field::operator== тем, что сравнивает значения разных числовых типов между собой.
|
||||
* Правила сравнения - такие же, что и в FunctionsComparison.
|
||||
* В том числе, сравнение знаковых и беззнаковых оставляем UB.
|
||||
*/
|
||||
class FieldVisitorAccurateEquals : public StaticVisitor<bool>
|
||||
{
|
||||
public:
|
||||
bool operator() (const Null & l, const Null & r) const { return true; }
|
||||
bool operator() (const Null & l, const UInt64 & r) const { return false; }
|
||||
bool operator() (const Null & l, const Int64 & r) const { return false; }
|
||||
bool operator() (const Null & l, const Float64 & r) const { return false; }
|
||||
bool operator() (const Null & l, const String & r) const { return false; }
|
||||
bool operator() (const Null & l, const Array & r) const { return false; }
|
||||
|
||||
bool operator() (const UInt64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const UInt64 & l, const UInt64 & r) const { return l == r; }
|
||||
bool operator() (const UInt64 & l, const Int64 & r) const { return l == r; }
|
||||
bool operator() (const UInt64 & l, const Float64 & r) const { return l == r; }
|
||||
bool operator() (const UInt64 & l, const String & r) const { return false; }
|
||||
bool operator() (const UInt64 & l, const Array & r) const { return false; }
|
||||
|
||||
bool operator() (const Int64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const Int64 & l, const UInt64 & r) const { return l == r; }
|
||||
bool operator() (const Int64 & l, const Int64 & r) const { return l == r; }
|
||||
bool operator() (const Int64 & l, const Float64 & r) const { return l == r; }
|
||||
bool operator() (const Int64 & l, const String & r) const { return false; }
|
||||
bool operator() (const Int64 & l, const Array & r) const { return false; }
|
||||
|
||||
bool operator() (const Float64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const Float64 & l, const UInt64 & r) const { return l == r; }
|
||||
bool operator() (const Float64 & l, const Int64 & r) const { return l == r; }
|
||||
bool operator() (const Float64 & l, const Float64 & r) const { return l == r; }
|
||||
bool operator() (const Float64 & l, const String & r) const { return false; }
|
||||
bool operator() (const Float64 & l, const Array & r) const { return false; }
|
||||
|
||||
bool operator() (const String & l, const Null & r) const { return false; }
|
||||
bool operator() (const String & l, const UInt64 & r) const { return false; }
|
||||
bool operator() (const String & l, const Int64 & r) const { return false; }
|
||||
bool operator() (const String & l, const Float64 & r) const { return false; }
|
||||
bool operator() (const String & l, const String & r) const { return l == r; }
|
||||
bool operator() (const String & l, const Array & r) const { return false; }
|
||||
|
||||
bool operator() (const Array & l, const Null & r) const { return false; }
|
||||
bool operator() (const Array & l, const UInt64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const Int64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const Float64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const String & r) const { return false; }
|
||||
bool operator() (const Array & l, const Array & r) const { return l == r; }
|
||||
};
|
||||
|
||||
class FieldVisitorAccurateLess : public StaticVisitor<bool>
|
||||
{
|
||||
public:
|
||||
bool operator() (const Null & l, const Null & r) const { return false; }
|
||||
bool operator() (const Null & l, const UInt64 & r) const { return true; }
|
||||
bool operator() (const Null & l, const Int64 & r) const { return true; }
|
||||
bool operator() (const Null & l, const Float64 & r) const { return true; }
|
||||
bool operator() (const Null & l, const String & r) const { return true; }
|
||||
bool operator() (const Null & l, const Array & r) const { return true; }
|
||||
|
||||
bool operator() (const UInt64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const UInt64 & l, const UInt64 & r) const { return l < r; }
|
||||
bool operator() (const UInt64 & l, const Int64 & r) const { return l < r; }
|
||||
bool operator() (const UInt64 & l, const Float64 & r) const { return l < r; }
|
||||
bool operator() (const UInt64 & l, const String & r) const { return true; }
|
||||
bool operator() (const UInt64 & l, const Array & r) const { return true; }
|
||||
|
||||
bool operator() (const Int64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const Int64 & l, const UInt64 & r) const { return l < r; }
|
||||
bool operator() (const Int64 & l, const Int64 & r) const { return l < r; }
|
||||
bool operator() (const Int64 & l, const Float64 & r) const { return l < r; }
|
||||
bool operator() (const Int64 & l, const String & r) const { return true; }
|
||||
bool operator() (const Int64 & l, const Array & r) const { return true; }
|
||||
|
||||
bool operator() (const Float64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const Float64 & l, const UInt64 & r) const { return l < r; }
|
||||
bool operator() (const Float64 & l, const Int64 & r) const { return l < r; }
|
||||
bool operator() (const Float64 & l, const Float64 & r) const { return l < r; }
|
||||
bool operator() (const Float64 & l, const String & r) const { return true; }
|
||||
bool operator() (const Float64 & l, const Array & r) const { return true; }
|
||||
|
||||
bool operator() (const String & l, const Null & r) const { return false; }
|
||||
bool operator() (const String & l, const UInt64 & r) const { return false; }
|
||||
bool operator() (const String & l, const Int64 & r) const { return false; }
|
||||
bool operator() (const String & l, const Float64 & r) const { return false; }
|
||||
bool operator() (const String & l, const String & r) const { return l < r; }
|
||||
bool operator() (const String & l, const Array & r) const { return true; }
|
||||
|
||||
bool operator() (const Array & l, const Null & r) const { return false; }
|
||||
bool operator() (const Array & l, const UInt64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const Int64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const Float64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const String & r) const { return false; }
|
||||
bool operator() (const Array & l, const Array & r) const { return l < r; }
|
||||
};
|
||||
|
||||
#pragma GCC diagnostic pop
|
||||
|
||||
/** Диапазон с открытыми или закрытыми концами; возможно, неограниченный.
|
||||
*/
|
||||
struct Range
|
||||
{
|
||||
private:
|
||||
static bool equals(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateEquals(), lhs, rhs); }
|
||||
static bool less(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateLess(), lhs, rhs); }
|
||||
static bool equals(const Field & lhs, const Field & rhs);
|
||||
static bool less(const Field & lhs, const Field & rhs);
|
||||
|
||||
public:
|
||||
Field left; /// левая граница, если есть
|
||||
|
@ -134,7 +134,7 @@ public:
|
||||
void getStatus(Status & res, bool with_zk_fields = true);
|
||||
|
||||
private:
|
||||
void dropUnreplicatedPartition(const Field & partition, const Settings & settings);
|
||||
void dropUnreplicatedPartition(const Field & partition, bool detach, const Settings & settings);
|
||||
|
||||
friend class ReplicatedMergeTreeBlockOutputStream;
|
||||
friend class ReplicatedMergeTreeRestartingThread;
|
||||
|
@ -96,6 +96,7 @@ private:
|
||||
};
|
||||
|
||||
bool is_interactive = true; /// Использовать readline интерфейс или batch режим.
|
||||
bool need_render_progress = true; /// Рисовать прогресс выполнения запроса.
|
||||
bool print_time_to_stderr = false; /// В неинтерактивном режиме, выводить время выполнения в stderr.
|
||||
bool stdin_is_not_tty = false; /// stdin - не терминал.
|
||||
|
||||
@ -268,6 +269,9 @@ private:
|
||||
insert_format = "Values";
|
||||
insert_format_max_block_size = config().getInt("insert_format_max_block_size", context.getSettingsRef().max_insert_block_size);
|
||||
|
||||
if (!is_interactive)
|
||||
need_render_progress = config().getBool("progress", false);
|
||||
|
||||
connect();
|
||||
|
||||
if (is_interactive)
|
||||
@ -850,11 +854,11 @@ private:
|
||||
/// Формат может быть указан в запросе.
|
||||
if (ASTQueryWithOutput * query_with_output = dynamic_cast<ASTQueryWithOutput *>(&*parsed_query))
|
||||
{
|
||||
if (query_with_output->format)
|
||||
if (query_with_output->getFormat() != nullptr)
|
||||
{
|
||||
if (has_vertical_output_suffix)
|
||||
throw Exception("Output format already specified", ErrorCodes::CLIENT_OUTPUT_FORMAT_SPECIFIED);
|
||||
if (ASTIdentifier * id = typeid_cast<ASTIdentifier *>(&*query_with_output->format))
|
||||
if (const ASTIdentifier * id = typeid_cast<const ASTIdentifier *>(query_with_output->getFormat()))
|
||||
current_format = id->name;
|
||||
}
|
||||
}
|
||||
@ -906,7 +910,7 @@ private:
|
||||
|
||||
void writeProgress()
|
||||
{
|
||||
if (!is_interactive)
|
||||
if (!need_render_progress)
|
||||
return;
|
||||
|
||||
static size_t increment = 0;
|
||||
@ -1053,6 +1057,7 @@ public:
|
||||
("vertical,E", "vertical output format, same as --format=Vertical or FORMAT Vertical or \\G at end of command")
|
||||
("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)")
|
||||
("stacktrace", "print stack traces of exceptions")
|
||||
("progress", "print progress even in non-interactive mode")
|
||||
APPLY_FOR_SETTINGS(DECLARE_SETTING)
|
||||
APPLY_FOR_LIMITS(DECLARE_LIMIT)
|
||||
;
|
||||
@ -1171,6 +1176,8 @@ public:
|
||||
config().setBool("vertical", true);
|
||||
if (options.count("stacktrace"))
|
||||
config().setBool("stacktrace", true);
|
||||
if (options.count("progress"))
|
||||
config().setBool("progress", true);
|
||||
if (options.count("time"))
|
||||
print_time_to_stderr = true;
|
||||
}
|
||||
|
@ -3,7 +3,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ParallelReplicas::ParallelReplicas(Connection * connection_, Settings * settings_, ThrottlerPtr throttler_)
|
||||
ParallelReplicas::ParallelReplicas(Connection * connection_, const Settings * settings_, ThrottlerPtr throttler_)
|
||||
: settings(settings_), throttler(throttler_),
|
||||
active_replica_count(1),
|
||||
supports_parallel_execution(false)
|
||||
@ -11,7 +11,7 @@ ParallelReplicas::ParallelReplicas(Connection * connection_, Settings * settings
|
||||
registerReplica(connection_);
|
||||
}
|
||||
|
||||
ParallelReplicas::ParallelReplicas(IConnectionPool * pool_, Settings * settings_, ThrottlerPtr throttler_)
|
||||
ParallelReplicas::ParallelReplicas(IConnectionPool * pool_, const Settings * settings_, ThrottlerPtr throttler_)
|
||||
: settings(settings_), throttler(throttler_)
|
||||
{
|
||||
if (pool_ == nullptr)
|
||||
@ -37,6 +37,7 @@ ParallelReplicas::ParallelReplicas(IConnectionPool * pool_, Settings * settings_
|
||||
supports_parallel_execution = false;
|
||||
|
||||
pool_entry = pool_->get(settings);
|
||||
if (!pool_entry.isNull())
|
||||
registerReplica(&*pool_entry);
|
||||
}
|
||||
}
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <DB/DataTypes/FieldToDataType.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/DataTypes/DataTypeFixedString.h>
|
||||
#include <DB/DataTypes/DataTypeArray.h>
|
||||
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
@ -61,7 +62,7 @@ template <> ColumnPtr ColumnConst<String>::convertToFullColumn() const
|
||||
}
|
||||
|
||||
|
||||
template <> ColumnPtr ColumnConst<Array>::convertToFullColumn() const
|
||||
ColumnPtr ColumnConst<Array>::convertToFullColumn() const
|
||||
{
|
||||
if (!data_type)
|
||||
throw Exception("No data type specified for ColumnConstArray", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -70,7 +71,8 @@ template <> ColumnPtr ColumnConst<Array>::convertToFullColumn() const
|
||||
if (!type)
|
||||
throw Exception("Non-array data type specified for ColumnConstArray", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
size_t array_size = data.size();
|
||||
const Array & array = getDataFromHolderImpl();
|
||||
size_t array_size = array.size();
|
||||
ColumnPtr nested_column = type->getNestedType()->createColumn();
|
||||
|
||||
ColumnArray * res = new ColumnArray(nested_column);
|
||||
@ -81,11 +83,27 @@ template <> ColumnPtr ColumnConst<Array>::convertToFullColumn() const
|
||||
{
|
||||
offsets[i] = (i + 1) * array_size;
|
||||
for (size_t j = 0; j < array_size; ++j)
|
||||
nested_column->insert(data[j]);
|
||||
nested_column->insert(array[j]);
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
StringRef ColumnConst<Array>::getDataAt(size_t n) const
|
||||
{
|
||||
throw Exception("Method getDataAt is not supported for " + this->getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
UInt64 ColumnConst<Array>::get64(size_t n) const
|
||||
{
|
||||
throw Exception("Method get64 is not supported for " + this->getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
StringRef ColumnConst<Array>::getDataAtWithTerminatingZero(size_t n) const
|
||||
{
|
||||
throw Exception("Method getDataAt is not supported for " + this->getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -59,6 +59,19 @@ void tryLogCurrentException(const char * log_name)
|
||||
|
||||
void tryLogCurrentException(Poco::Logger * logger)
|
||||
{
|
||||
try
|
||||
{
|
||||
LOG_ERROR(logger, getCurrentExceptionMessage(true));
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
}
|
||||
}
|
||||
|
||||
std::string getCurrentExceptionMessage(bool with_stacktrace)
|
||||
{
|
||||
std::stringstream stream;
|
||||
|
||||
try
|
||||
{
|
||||
throw;
|
||||
@ -67,8 +80,10 @@ void tryLogCurrentException(Poco::Logger * logger)
|
||||
{
|
||||
try
|
||||
{
|
||||
LOG_ERROR(logger, "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what()
|
||||
<< ", Stack trace:\n\n" << e.getStackTrace().toString());
|
||||
stream << "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what();
|
||||
|
||||
if (with_stacktrace)
|
||||
stream << ", Stack trace:\n\n" << e.getStackTrace().toString();
|
||||
}
|
||||
catch (...) {}
|
||||
}
|
||||
@ -76,8 +91,8 @@ void tryLogCurrentException(Poco::Logger * logger)
|
||||
{
|
||||
try
|
||||
{
|
||||
LOG_ERROR(logger, "Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
|
||||
<< ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what());
|
||||
stream << "Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
|
||||
<< ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what();
|
||||
}
|
||||
catch (...) {}
|
||||
}
|
||||
@ -91,7 +106,7 @@ void tryLogCurrentException(Poco::Logger * logger)
|
||||
if (status)
|
||||
name += " (demangling status: " + toString(status) + ")";
|
||||
|
||||
LOG_ERROR(logger, "std::exception. Code: " << ErrorCodes::STD_EXCEPTION << ", type: " << name << ", e.what() = " << e.what());
|
||||
stream << "std::exception. Code: " << ErrorCodes::STD_EXCEPTION << ", type: " << name << ", e.what() = " << e.what();
|
||||
}
|
||||
catch (...) {}
|
||||
}
|
||||
@ -105,10 +120,12 @@ void tryLogCurrentException(Poco::Logger * logger)
|
||||
if (status)
|
||||
name += " (demangling status: " + toString(status) + ")";
|
||||
|
||||
LOG_ERROR(logger, "Unknown exception. Code: " << ErrorCodes::UNKNOWN_EXCEPTION << ", type: " << name);
|
||||
stream << "Unknown exception. Code: " << ErrorCodes::UNKNOWN_EXCEPTION << ", type: " << name;
|
||||
}
|
||||
catch (...) {}
|
||||
}
|
||||
|
||||
return stream.str();
|
||||
}
|
||||
|
||||
|
||||
|
9
dbms/src/DataStreams/BlockIO.cpp
Normal file
9
dbms/src/DataStreams/BlockIO.cpp
Normal file
@ -0,0 +1,9 @@
|
||||
#include <DB/Interpreters/ProcessList.h>
|
||||
#include <DB/DataStreams/BlockIO.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
BlockIO::~BlockIO() = default;
|
||||
|
||||
}
|
@ -5,6 +5,7 @@
|
||||
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
#include <DB/Interpreters/Quota.h>
|
||||
#include <DB/Interpreters/ProcessList.h>
|
||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||
|
||||
|
||||
@ -320,7 +321,7 @@ void IProfilingBlockInputStream::setProgressCallback(ProgressCallback callback)
|
||||
}
|
||||
|
||||
|
||||
void IProfilingBlockInputStream::setProcessListElement(ProcessList::Element * elem)
|
||||
void IProfilingBlockInputStream::setProcessListElement(ProcessListElement * elem)
|
||||
{
|
||||
process_list_elem = elem;
|
||||
|
||||
|
170
dbms/src/DataTypes/FieldToDataType.cpp
Normal file
170
dbms/src/DataTypes/FieldToDataType.cpp
Normal file
@ -0,0 +1,170 @@
|
||||
#include <DB/DataTypes/FieldToDataType.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
template <typename T>
|
||||
static void convertArrayToCommonType(Array & arr)
|
||||
{
|
||||
for (auto & elem : arr)
|
||||
elem = apply_visitor(FieldVisitorConvertToNumber<T>(), elem);
|
||||
}
|
||||
|
||||
|
||||
DataTypePtr FieldToDataType::operator() (Array & x) const
|
||||
{
|
||||
if (x.empty())
|
||||
throw Exception("Cannot infer type of empty array", ErrorCodes::EMPTY_DATA_PASSED);
|
||||
|
||||
/** Тип массива нужно вывести по типу его элементов.
|
||||
* Если элементы - числа, то нужно выбрать наименьший общий тип, если такой есть,
|
||||
* или кинуть исключение.
|
||||
* Код похож на NumberTraits::ResultOfIf, но тем кодом трудно здесь непосредственно воспользоваться.
|
||||
*
|
||||
* Также заметим, что Float32 не выводится, вместо этого используется только Float64.
|
||||
* Это сделано потому что литералов типа Float32 не бывает в запросе.
|
||||
*/
|
||||
|
||||
bool has_string = false;
|
||||
bool has_array = false;
|
||||
bool has_float = false;
|
||||
int max_bits = 0;
|
||||
int max_signed_bits = 0;
|
||||
int max_unsigned_bits = 0;
|
||||
|
||||
for (const Field & elem : x)
|
||||
{
|
||||
switch (elem.getType())
|
||||
{
|
||||
case Field::Types::UInt64:
|
||||
{
|
||||
UInt64 num = elem.get<UInt64>();
|
||||
if (num <= std::numeric_limits<UInt8>::max())
|
||||
max_unsigned_bits = std::max(8, max_unsigned_bits);
|
||||
else if (num <= std::numeric_limits<UInt16>::max())
|
||||
max_unsigned_bits = std::max(16, max_unsigned_bits);
|
||||
else if (num <= std::numeric_limits<UInt32>::max())
|
||||
max_unsigned_bits = std::max(32, max_unsigned_bits);
|
||||
else
|
||||
max_unsigned_bits = 64;
|
||||
max_bits = std::max(max_unsigned_bits, max_bits);
|
||||
break;
|
||||
}
|
||||
case Field::Types::Int64:
|
||||
{
|
||||
Int64 num = elem.get<Int64>();
|
||||
if (num <= std::numeric_limits<Int8>::max() && num >= std::numeric_limits<Int8>::min())
|
||||
max_signed_bits = std::max(8, max_signed_bits);
|
||||
else if (num <= std::numeric_limits<Int16>::max() && num >= std::numeric_limits<Int16>::min())
|
||||
max_signed_bits = std::max(16, max_signed_bits);
|
||||
else if (num <= std::numeric_limits<Int32>::max() && num >= std::numeric_limits<Int32>::min())
|
||||
max_signed_bits = std::max(32, max_signed_bits);
|
||||
else
|
||||
max_signed_bits = 64;
|
||||
max_bits = std::max(max_signed_bits, max_bits);
|
||||
break;
|
||||
}
|
||||
case Field::Types::Float64:
|
||||
{
|
||||
has_float = true;
|
||||
break;
|
||||
}
|
||||
case Field::Types::String:
|
||||
{
|
||||
has_string = true;
|
||||
break;
|
||||
}
|
||||
case Field::Types::Array:
|
||||
{
|
||||
has_array = true;
|
||||
break;
|
||||
}
|
||||
case Field::Types::Null:
|
||||
{
|
||||
throw Exception("NULL literals are not implemented yet", ErrorCodes::NOT_IMPLEMENTED);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if ((has_string + has_array + (max_bits > 0)) > 1)
|
||||
throw Exception("Incompatible types of elements of array", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (has_array)
|
||||
throw Exception("Type inference of multidimensional arrays is not supported", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (has_string)
|
||||
return new DataTypeArray(new DataTypeString);
|
||||
|
||||
if (has_float && max_bits == 64)
|
||||
throw Exception("Incompatible types Float64 and UInt64/Int64 of elements of array", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (has_float)
|
||||
{
|
||||
convertArrayToCommonType<Float64>(x);
|
||||
return new DataTypeArray(new DataTypeFloat64);
|
||||
}
|
||||
|
||||
if (max_signed_bits == 64 && max_unsigned_bits == 64)
|
||||
throw Exception("Incompatible types UInt64 and Int64 of elements of array", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (max_signed_bits && !max_unsigned_bits)
|
||||
{
|
||||
if (max_signed_bits == 8)
|
||||
return new DataTypeArray(new DataTypeInt8);
|
||||
if (max_signed_bits == 16)
|
||||
return new DataTypeArray(new DataTypeInt16);
|
||||
if (max_signed_bits == 32)
|
||||
return new DataTypeArray(new DataTypeInt32);
|
||||
if (max_signed_bits == 64)
|
||||
return new DataTypeArray(new DataTypeInt64);
|
||||
}
|
||||
|
||||
if (!max_signed_bits && max_unsigned_bits)
|
||||
{
|
||||
if (max_unsigned_bits == 8)
|
||||
return new DataTypeArray(new DataTypeUInt8);
|
||||
if (max_unsigned_bits == 16)
|
||||
return new DataTypeArray(new DataTypeUInt16);
|
||||
if (max_unsigned_bits == 32)
|
||||
return new DataTypeArray(new DataTypeUInt32);
|
||||
if (max_unsigned_bits == 64)
|
||||
return new DataTypeArray(new DataTypeUInt64);
|
||||
}
|
||||
|
||||
if (max_signed_bits && max_unsigned_bits)
|
||||
{
|
||||
convertArrayToCommonType<Int64>(x);
|
||||
|
||||
if (max_unsigned_bits >= max_signed_bits)
|
||||
{
|
||||
/// Беззнаковый тип не помещается в знаковый. Надо увеличить количество бит.
|
||||
if (max_bits == 8)
|
||||
return new DataTypeArray(new DataTypeInt16);
|
||||
if (max_bits == 16)
|
||||
return new DataTypeArray(new DataTypeInt32);
|
||||
if (max_bits == 32)
|
||||
return new DataTypeArray(new DataTypeInt64);
|
||||
else
|
||||
throw Exception("Incompatible types UInt64 and signed integer of elements of array", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Беззнаковый тип помещается в знаковый.
|
||||
if (max_bits == 8)
|
||||
return new DataTypeArray(new DataTypeInt8);
|
||||
if (max_bits == 16)
|
||||
return new DataTypeArray(new DataTypeInt16);
|
||||
if (max_bits == 32)
|
||||
return new DataTypeArray(new DataTypeInt32);
|
||||
if (max_bits == 64)
|
||||
return new DataTypeArray(new DataTypeInt64);
|
||||
}
|
||||
}
|
||||
|
||||
throw Exception("Incompatible types of elements of array", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
|
||||
}
|
@ -326,6 +326,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionMaterialize>();
|
||||
factory.registerFunction<FunctionIgnore>();
|
||||
factory.registerFunction<FunctionArrayJoin>();
|
||||
factory.registerFunction<FunctionReplicate>();
|
||||
factory.registerFunction<FunctionBar>();
|
||||
|
||||
factory.registerFunction<FunctionTuple>();
|
||||
|
@ -36,6 +36,16 @@ void assertString(const char * s, ReadBuffer & buf)
|
||||
}
|
||||
}
|
||||
|
||||
void assertChar(char symbol, ReadBuffer & buf)
|
||||
{
|
||||
if (buf.eof() || *buf.position() != symbol)
|
||||
{
|
||||
char err[2] = {symbol, '\0'};
|
||||
throwAtAssertionFailed(err, buf);
|
||||
}
|
||||
++buf.position();
|
||||
}
|
||||
|
||||
void assertEOF(ReadBuffer & buf)
|
||||
{
|
||||
if (!buf.eof())
|
||||
|
@ -557,14 +557,21 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
|
||||
for (size_t i = 0; i < aggregates_size; ++i)
|
||||
aggregate_columns[i].resize(aggregates[i].arguments.size());
|
||||
|
||||
/** Константные столбцы не поддерживаются напрямую при агрегации.
|
||||
* Чтобы они всё-равно работали, материализуем их.
|
||||
*/
|
||||
Columns materialized_columns;
|
||||
|
||||
/// Запоминаем столбцы, с которыми будем работать
|
||||
for (size_t i = 0; i < keys_size; ++i)
|
||||
{
|
||||
key_columns[i] = block.getByPosition(keys[i]).column;
|
||||
|
||||
if (key_columns[i]->isConst())
|
||||
throw Exception("Constants are not allowed as GROUP BY keys"
|
||||
" (but all of them must be eliminated in ExpressionAnalyzer)", ErrorCodes::ILLEGAL_COLUMN);
|
||||
if (const IColumnConst * column_const = dynamic_cast<const IColumnConst *>(key_columns[i]))
|
||||
{
|
||||
materialized_columns.push_back(column_const->convertToFullColumn());
|
||||
key_columns[i] = materialized_columns.back().get();
|
||||
}
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < aggregates_size; ++i)
|
||||
@ -573,11 +580,11 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
|
||||
{
|
||||
aggregate_columns[i][j] = block.getByPosition(aggregates[i].arguments[j]).column;
|
||||
|
||||
/** Агрегатные функции рассчитывают, что в них передаются полноценные столбцы.
|
||||
* Поэтому, стобцы-константы не разрешены в качестве аргументов агрегатных функций.
|
||||
*/
|
||||
if (aggregate_columns[i][j]->isConst())
|
||||
throw Exception("Constants are not allowed as arguments of aggregate functions", ErrorCodes::ILLEGAL_COLUMN);
|
||||
if (const IColumnConst * column_const = dynamic_cast<const IColumnConst *>(aggregate_columns[i][j]))
|
||||
{
|
||||
materialized_columns.push_back(column_const->convertToFullColumn());
|
||||
aggregate_columns[i][j] = materialized_columns.back().get();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -27,6 +27,7 @@
|
||||
#include <DB/Interpreters/Cluster.h>
|
||||
#include <DB/Interpreters/InterserverIOHandler.h>
|
||||
#include <DB/Interpreters/Compiler.h>
|
||||
#include <DB/Interpreters/QueryLog.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/IO/ReadBufferFromFile.h>
|
||||
#include <DB/IO/WriteBufferFromString.h>
|
||||
@ -86,6 +87,7 @@ struct ContextShared
|
||||
BackgroundProcessingPoolPtr background_pool; /// Пул потоков для фоновой работы, выполняемой таблицами.
|
||||
Macros macros; /// Подстановки из конфига.
|
||||
std::unique_ptr<Compiler> compiler; /// Для динамической компиляции частей запроса, при необходимости.
|
||||
std::unique_ptr<QueryLog> query_log; /// Для логгирования запросов.
|
||||
mutable std::unique_ptr<CompressionMethodSelector> compression_method_selector; /// Правила для выбора метода сжатия в зависимости от размера куска.
|
||||
|
||||
/// Кластеры для distributed таблиц
|
||||
@ -830,6 +832,26 @@ Compiler & Context::getCompiler()
|
||||
}
|
||||
|
||||
|
||||
QueryLog & Context::getQueryLog()
|
||||
{
|
||||
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
|
||||
|
||||
if (!shared->query_log)
|
||||
{
|
||||
auto & config = Poco::Util::Application::instance().config();
|
||||
|
||||
String database = config.getString("query_log.database", "system");
|
||||
String table = config.getString("query_log.table", "query_log");
|
||||
size_t flush_interval_milliseconds = parse<size_t>(
|
||||
config.getString("query_log.flush_interval_milliseconds", DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS_STR));
|
||||
|
||||
shared->query_log.reset(new QueryLog{ *this, database, table, flush_interval_milliseconds });
|
||||
}
|
||||
|
||||
return *shared->query_log;
|
||||
}
|
||||
|
||||
|
||||
CompressionMethod Context::chooseCompressionMethod(size_t part_size, double part_size_ratio) const
|
||||
{
|
||||
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
|
||||
|
@ -768,7 +768,7 @@ void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name)
|
||||
StoragePtr external_storage = StorageMemory::create(external_table_name, columns);
|
||||
|
||||
external_tables[external_table_name] = external_storage;
|
||||
subqueries_for_sets[external_table_name].source = interpreter->execute();
|
||||
subqueries_for_sets[external_table_name].source = interpreter->execute().in;
|
||||
subqueries_for_sets[external_table_name].source_sample = interpreter->getSampleBlock();
|
||||
subqueries_for_sets[external_table_name].table = external_storage;
|
||||
|
||||
@ -842,7 +842,7 @@ void ExpressionAnalyzer::makeSet(ASTFunction * node, const Block & sample_block)
|
||||
if (!subquery_for_set.source)
|
||||
{
|
||||
auto interpreter = interpretSubquery(arg, context, subquery_depth);
|
||||
subquery_for_set.source = new LazyBlockInputStream([interpreter]() mutable { return interpreter->execute(); });
|
||||
subquery_for_set.source = new LazyBlockInputStream([interpreter]() mutable { return interpreter->execute().in; });
|
||||
subquery_for_set.source_sample = interpreter->getSampleBlock();
|
||||
|
||||
/** Зачем используется LazyBlockInputStream?
|
||||
@ -1399,6 +1399,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl
|
||||
else if (ASTLiteral * node = typeid_cast<ASTLiteral *>(&*ast))
|
||||
{
|
||||
DataTypePtr type = apply_visitor(FieldToDataType(), node->value);
|
||||
|
||||
ColumnWithNameAndType column;
|
||||
column.column = type->createConstColumn(1, node->value);
|
||||
column.type = type;
|
||||
@ -1594,7 +1595,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
|
||||
if (!subquery_for_set.source)
|
||||
{
|
||||
auto interpreter = interpretSubquery(ast_join.table, context, subquery_depth, required_joined_columns);
|
||||
subquery_for_set.source = new LazyBlockInputStream([interpreter]() mutable { return interpreter->execute(); });
|
||||
subquery_for_set.source = new LazyBlockInputStream([interpreter]() mutable { return interpreter->execute().in; });
|
||||
subquery_for_set.source_sample = interpreter->getSampleBlock();
|
||||
}
|
||||
|
||||
|
@ -81,6 +81,10 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error)
|
||||
{
|
||||
const std::lock_guard<std::mutex> lock{dictionaries_mutex};
|
||||
|
||||
const auto & lifetime = dict_ptr->getLifetime();
|
||||
std::uniform_int_distribution<std::uint64_t> distribution{lifetime.min_sec, lifetime.max_sec};
|
||||
update_times[name] = std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)};
|
||||
|
||||
const auto dict_it = dictionaries.find(name);
|
||||
if (dict_it->second.dict)
|
||||
dict_it->second.dict->set(dict_ptr.release());
|
||||
@ -90,10 +94,6 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error)
|
||||
/// erase stored exception on success
|
||||
dict_it->second.exception = std::exception_ptr{};
|
||||
|
||||
const auto & lifetime = dict_ptr->getLifetime();
|
||||
std::uniform_int_distribution<std::uint64_t> distribution{lifetime.min_sec, lifetime.max_sec};
|
||||
update_times[name] = std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)};
|
||||
|
||||
recreated_failed_dictionaries.push_back(name);
|
||||
}
|
||||
}
|
||||
|
@ -27,7 +27,7 @@ InterpreterAlterQuery::InterpreterAlterQuery(ASTPtr query_ptr_, Context & contex
|
||||
{
|
||||
}
|
||||
|
||||
void InterpreterAlterQuery::execute()
|
||||
BlockIO InterpreterAlterQuery::execute()
|
||||
{
|
||||
auto & alter = typeid_cast<ASTAlterQuery &>(*query_ptr);
|
||||
const String & table_name = alter.table;
|
||||
@ -64,11 +64,13 @@ void InterpreterAlterQuery::execute()
|
||||
}
|
||||
|
||||
if (alter_commands.empty())
|
||||
return;
|
||||
return {};
|
||||
|
||||
alter_commands.validate(table.get(), context);
|
||||
|
||||
table->alter(alter_commands, database_name, table_name, context);
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
void InterpreterAlterQuery::parseAlter(
|
||||
|
@ -4,13 +4,14 @@
|
||||
#include <DB/Columns/ColumnsNumber.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
|
||||
using namespace DB;
|
||||
namespace DB
|
||||
{
|
||||
|
||||
InterpreterCheckQuery::InterpreterCheckQuery(DB::ASTPtr query_ptr_, DB::Context& context_) : query_ptr(query_ptr_), context(context_)
|
||||
{
|
||||
}
|
||||
|
||||
BlockInputStreamPtr InterpreterCheckQuery::execute()
|
||||
BlockIO InterpreterCheckQuery::execute()
|
||||
{
|
||||
ASTCheckQuery & alter = typeid_cast<ASTCheckQuery &>(*query_ptr);
|
||||
String & table_name = alter.table;
|
||||
@ -18,16 +19,14 @@ BlockInputStreamPtr InterpreterCheckQuery::execute()
|
||||
|
||||
StoragePtr table = context.getTable(database_name, table_name);
|
||||
|
||||
result = getSampleBlock();
|
||||
result = Block{{ new ColumnUInt8, new DataTypeUInt8, "result" }};
|
||||
result.getByPosition(0).column->insert(Field(UInt64(table->checkData())));
|
||||
|
||||
return BlockInputStreamPtr(new OneBlockInputStream(result));
|
||||
BlockIO res;
|
||||
res.in = new OneBlockInputStream(result);
|
||||
res.in_sample = result.cloneEmpty();
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
Block InterpreterCheckQuery::getSampleBlock()
|
||||
{
|
||||
DB::Block b;
|
||||
ColumnPtr column(new ColumnUInt8);
|
||||
b.insert(ColumnWithNameAndType(column, new DataTypeUInt8, "result"));
|
||||
return b;
|
||||
}
|
||||
|
@ -42,7 +42,7 @@ InterpreterCreateQuery::InterpreterCreateQuery(ASTPtr query_ptr_, Context & cont
|
||||
}
|
||||
|
||||
|
||||
StoragePtr InterpreterCreateQuery::execute(bool assume_metadata_exists)
|
||||
void InterpreterCreateQuery::executeImpl(bool assume_metadata_exists)
|
||||
{
|
||||
String path = context.getPath();
|
||||
String current_database = context.getCurrentDatabase();
|
||||
@ -81,7 +81,7 @@ StoragePtr InterpreterCreateQuery::execute(bool assume_metadata_exists)
|
||||
if (!create.if_not_exists || !context.isDatabaseExist(database_name))
|
||||
context.addDatabase(database_name);
|
||||
|
||||
return StoragePtr();
|
||||
return;
|
||||
}
|
||||
|
||||
SharedPtr<InterpreterSelectQuery> interpreter_select;
|
||||
@ -118,7 +118,7 @@ StoragePtr InterpreterCreateQuery::execute(bool assume_metadata_exists)
|
||||
if (context.isTableExist(database_name, table_name))
|
||||
{
|
||||
if (create.if_not_exists)
|
||||
return context.getTable(database_name, table_name);
|
||||
return;
|
||||
else
|
||||
throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
}
|
||||
@ -251,11 +251,9 @@ StoragePtr InterpreterCreateQuery::execute(bool assume_metadata_exists)
|
||||
/// Если запрос CREATE SELECT, то вставим в таблицу данные
|
||||
if (create.select && storage_name != "View" && (storage_name != "MaterializedView" || create.is_populate))
|
||||
{
|
||||
BlockInputStreamPtr from = new MaterializingBlockInputStream(interpreter_select->execute());
|
||||
BlockInputStreamPtr from = new MaterializingBlockInputStream(interpreter_select->execute().in);
|
||||
copyData(*from, *res->write(query_ptr));
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
InterpreterCreateQuery::ColumnsAndDefaults InterpreterCreateQuery::parseColumns(ASTPtr expression_list)
|
||||
|
@ -16,7 +16,7 @@ InterpreterDropQuery::InterpreterDropQuery(ASTPtr query_ptr_, Context & context_
|
||||
}
|
||||
|
||||
|
||||
void InterpreterDropQuery::execute()
|
||||
BlockIO InterpreterDropQuery::execute()
|
||||
{
|
||||
String path = context.getPath();
|
||||
String current_database = context.getCurrentDatabase();
|
||||
@ -43,7 +43,7 @@ void InterpreterDropQuery::execute()
|
||||
if (table)
|
||||
tables_to_drop.push_back(table);
|
||||
else
|
||||
return;
|
||||
return {};
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -52,7 +52,7 @@ void InterpreterDropQuery::execute()
|
||||
if (!drop.if_exists)
|
||||
context.assertDatabaseExists(database_name);
|
||||
else if (!context.isDatabaseExist(database_name))
|
||||
return;
|
||||
return {};
|
||||
|
||||
Tables tables = context.getDatabases()[database_name];
|
||||
|
||||
@ -111,6 +111,8 @@ void InterpreterDropQuery::execute()
|
||||
Poco::File(data_path).remove(false);
|
||||
Poco::File(metadata_path).remove(false);
|
||||
}
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
void InterpreterDropQuery::dropDetachedTable(String database_name, StoragePtr table, Context & context)
|
||||
|
116
dbms/src/Interpreters/InterpreterFactory.cpp
Normal file
116
dbms/src/Interpreters/InterpreterFactory.cpp
Normal file
@ -0,0 +1,116 @@
|
||||
#include <DB/Parsers/ASTInsertQuery.h>
|
||||
#include <DB/Parsers/ASTSelectQuery.h>
|
||||
#include <DB/Parsers/ASTCreateQuery.h>
|
||||
#include <DB/Parsers/ASTDropQuery.h>
|
||||
#include <DB/Parsers/ASTRenameQuery.h>
|
||||
#include <DB/Parsers/ASTShowTablesQuery.h>
|
||||
#include <DB/Parsers/ASTUseQuery.h>
|
||||
#include <DB/Parsers/ASTSetQuery.h>
|
||||
#include <DB/Parsers/ASTOptimizeQuery.h>
|
||||
#include <DB/Parsers/ASTAlterQuery.h>
|
||||
#include <DB/Parsers/ASTShowProcesslistQuery.h>
|
||||
#include <DB/Parsers/TablePropertiesQueriesASTs.h>
|
||||
#include <DB/Parsers/ASTCheckQuery.h>
|
||||
|
||||
#include <DB/Interpreters/InterpreterSelectQuery.h>
|
||||
#include <DB/Interpreters/InterpreterInsertQuery.h>
|
||||
#include <DB/Interpreters/InterpreterCreateQuery.h>
|
||||
#include <DB/Interpreters/InterpreterDropQuery.h>
|
||||
#include <DB/Interpreters/InterpreterRenameQuery.h>
|
||||
#include <DB/Interpreters/InterpreterShowTablesQuery.h>
|
||||
#include <DB/Interpreters/InterpreterUseQuery.h>
|
||||
#include <DB/Interpreters/InterpreterSetQuery.h>
|
||||
#include <DB/Interpreters/InterpreterOptimizeQuery.h>
|
||||
#include <DB/Interpreters/InterpreterExistsQuery.h>
|
||||
#include <DB/Interpreters/InterpreterDescribeQuery.h>
|
||||
#include <DB/Interpreters/InterpreterShowCreateQuery.h>
|
||||
#include <DB/Interpreters/InterpreterAlterQuery.h>
|
||||
#include <DB/Interpreters/InterpreterShowProcesslistQuery.h>
|
||||
#include <DB/Interpreters/InterpreterCheckQuery.h>
|
||||
#include <DB/Interpreters/InterpreterFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
static void throwIfReadOnly(Context & context)
|
||||
{
|
||||
if (context.getSettingsRef().limits.readonly)
|
||||
throw Exception("Cannot execute query in readonly mode", ErrorCodes::READONLY);
|
||||
}
|
||||
|
||||
|
||||
SharedPtr<IInterpreter> InterpreterFactory::get(ASTPtr & query, Context & context, QueryProcessingStage::Enum stage)
|
||||
{
|
||||
if (typeid_cast<ASTSelectQuery *>(query.get()))
|
||||
{
|
||||
return new InterpreterSelectQuery(query, context, stage);
|
||||
}
|
||||
else if (typeid_cast<ASTInsertQuery *>(query.get()))
|
||||
{
|
||||
throwIfReadOnly(context);
|
||||
return new InterpreterInsertQuery(query, context);
|
||||
}
|
||||
else if (typeid_cast<ASTCreateQuery *>(query.get()))
|
||||
{
|
||||
throwIfReadOnly(context);
|
||||
return new InterpreterCreateQuery(query, context);
|
||||
}
|
||||
else if (typeid_cast<ASTDropQuery *>(query.get()))
|
||||
{
|
||||
throwIfReadOnly(context);
|
||||
return new InterpreterDropQuery(query, context);
|
||||
}
|
||||
else if (typeid_cast<ASTRenameQuery *>(query.get()))
|
||||
{
|
||||
throwIfReadOnly(context);
|
||||
return new InterpreterRenameQuery(query, context);
|
||||
}
|
||||
else if (typeid_cast<ASTShowTablesQuery *>(query.get()))
|
||||
{
|
||||
return new InterpreterShowTablesQuery(query, context);
|
||||
}
|
||||
else if (typeid_cast<ASTUseQuery *>(query.get()))
|
||||
{
|
||||
return new InterpreterUseQuery(query, context);
|
||||
}
|
||||
else if (typeid_cast<ASTSetQuery *>(query.get()))
|
||||
{
|
||||
/// readonly проверяется внутри InterpreterSetQuery
|
||||
return new InterpreterSetQuery(query, context);
|
||||
}
|
||||
else if (typeid_cast<ASTOptimizeQuery *>(query.get()))
|
||||
{
|
||||
throwIfReadOnly(context);
|
||||
return new InterpreterOptimizeQuery(query, context);
|
||||
}
|
||||
else if (typeid_cast<ASTExistsQuery *>(query.get()))
|
||||
{
|
||||
return new InterpreterExistsQuery(query, context);
|
||||
}
|
||||
else if (typeid_cast<ASTShowCreateQuery *>(query.get()))
|
||||
{
|
||||
return new InterpreterShowCreateQuery(query, context);
|
||||
}
|
||||
else if (typeid_cast<ASTDescribeQuery *>(query.get()))
|
||||
{
|
||||
return new InterpreterDescribeQuery(query, context);
|
||||
}
|
||||
else if (typeid_cast<ASTShowProcesslistQuery *>(query.get()))
|
||||
{
|
||||
return new InterpreterShowProcesslistQuery(query, context);
|
||||
}
|
||||
else if (typeid_cast<ASTAlterQuery *>(query.get()))
|
||||
{
|
||||
throwIfReadOnly(context);
|
||||
return new InterpreterAlterQuery(query, context);
|
||||
}
|
||||
else if (typeid_cast<ASTCheckQuery *>(query.get()))
|
||||
{
|
||||
return new InterpreterCheckQuery(query, context);
|
||||
}
|
||||
else
|
||||
throw Exception("Unknown type of query: " + query->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY);
|
||||
}
|
||||
|
||||
}
|
@ -64,69 +64,6 @@ Block InterpreterInsertQuery::getSampleBlock()
|
||||
return res;
|
||||
}
|
||||
|
||||
void InterpreterInsertQuery::execute(ReadBuffer * remaining_data_istr)
|
||||
{
|
||||
ASTInsertQuery & query = typeid_cast<ASTInsertQuery &>(*query_ptr);
|
||||
StoragePtr table = getTable();
|
||||
|
||||
auto table_lock = table->lockStructure(true);
|
||||
|
||||
/** @note looks suspicious, first we ask to create block from NamesAndTypesList (internally in ITableDeclaration),
|
||||
* then we compose the same list from the resulting block */
|
||||
NamesAndTypesListPtr required_columns = new NamesAndTypesList(table->getColumnsList());
|
||||
|
||||
/// Создаем кортеж из нескольких стримов, в которые будем писать данные.
|
||||
BlockOutputStreamPtr out{
|
||||
new ProhibitColumnsBlockOutputStream{
|
||||
new AddingDefaultBlockOutputStream{
|
||||
new MaterializingBlockOutputStream{
|
||||
new PushingToViewsBlockOutputStream{query.database, query.table, context, query_ptr}
|
||||
},
|
||||
required_columns, table->column_defaults, context, context.getSettingsRef().strict_insert_defaults
|
||||
},
|
||||
table->materialized_columns
|
||||
}
|
||||
};
|
||||
|
||||
/// Какой тип запроса: INSERT VALUES | INSERT FORMAT | INSERT SELECT?
|
||||
if (!query.select)
|
||||
{
|
||||
|
||||
String format = query.format;
|
||||
if (format.empty())
|
||||
format = "Values";
|
||||
|
||||
/// Данные могут содержаться в распарсенной (query.data) и ещё не распарсенной (remaining_data_istr) части запроса.
|
||||
|
||||
ConcatReadBuffer::ReadBuffers buffers;
|
||||
ReadBuffer buf1(const_cast<char *>(query.data), query.data ? query.end - query.data : 0, 0);
|
||||
|
||||
if (query.data)
|
||||
buffers.push_back(&buf1);
|
||||
buffers.push_back(remaining_data_istr);
|
||||
|
||||
/** NOTE Нельзя читать из remaining_data_istr до того, как прочтём всё между query.data и query.end.
|
||||
* - потому что query.data может ссылаться на кусок памяти, использующийся в качестве буфера в remaining_data_istr.
|
||||
*/
|
||||
|
||||
ConcatReadBuffer istr(buffers);
|
||||
Block sample = getSampleBlock();
|
||||
|
||||
BlockInputStreamPtr in{
|
||||
context.getFormatFactory().getInput(
|
||||
format, istr, sample, context.getSettings().max_insert_block_size)};
|
||||
|
||||
copyData(*in, *out);
|
||||
}
|
||||
else
|
||||
{
|
||||
InterpreterSelectQuery interpreter_select(query.select, context);
|
||||
BlockInputStreamPtr in{interpreter_select.execute()};
|
||||
|
||||
copyData(*in, *out);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
BlockIO InterpreterInsertQuery::execute()
|
||||
{
|
||||
@ -161,7 +98,7 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
else
|
||||
{
|
||||
InterpreterSelectQuery interpreter_select{query.select, context};
|
||||
BlockInputStreamPtr in{interpreter_select.execute()};
|
||||
BlockInputStreamPtr in{interpreter_select.execute().in};
|
||||
res.in = new NullAndDoCopyBlockInputStream{in, out};
|
||||
}
|
||||
|
||||
|
@ -1,227 +0,0 @@
|
||||
#include <DB/Parsers/ASTInsertQuery.h>
|
||||
#include <DB/Parsers/ASTSelectQuery.h>
|
||||
#include <DB/Parsers/ASTCreateQuery.h>
|
||||
#include <DB/Parsers/ASTDropQuery.h>
|
||||
#include <DB/Parsers/ASTRenameQuery.h>
|
||||
#include <DB/Parsers/ASTShowTablesQuery.h>
|
||||
#include <DB/Parsers/ASTUseQuery.h>
|
||||
#include <DB/Parsers/ASTSetQuery.h>
|
||||
#include <DB/Parsers/ASTOptimizeQuery.h>
|
||||
#include <DB/Parsers/ASTAlterQuery.h>
|
||||
#include <DB/Parsers/ASTShowProcesslistQuery.h>
|
||||
#include <DB/Parsers/TablePropertiesQueriesASTs.h>
|
||||
#include <DB/Parsers/ASTCheckQuery.h>
|
||||
|
||||
#include <DB/Interpreters/InterpreterSelectQuery.h>
|
||||
#include <DB/Interpreters/InterpreterInsertQuery.h>
|
||||
#include <DB/Interpreters/InterpreterCreateQuery.h>
|
||||
#include <DB/Interpreters/InterpreterDropQuery.h>
|
||||
#include <DB/Interpreters/InterpreterRenameQuery.h>
|
||||
#include <DB/Interpreters/InterpreterShowTablesQuery.h>
|
||||
#include <DB/Interpreters/InterpreterUseQuery.h>
|
||||
#include <DB/Interpreters/InterpreterSetQuery.h>
|
||||
#include <DB/Interpreters/InterpreterOptimizeQuery.h>
|
||||
#include <DB/Interpreters/InterpreterExistsQuery.h>
|
||||
#include <DB/Interpreters/InterpreterDescribeQuery.h>
|
||||
#include <DB/Interpreters/InterpreterShowCreateQuery.h>
|
||||
#include <DB/Interpreters/InterpreterQuery.h>
|
||||
#include <DB/Interpreters/InterpreterAlterQuery.h>
|
||||
#include <DB/Interpreters/InterpreterShowProcesslistQuery.h>
|
||||
#include <DB/Interpreters/InterpreterCheckQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
InterpreterQuery::InterpreterQuery(ASTPtr query_ptr_, Context & context_, QueryProcessingStage::Enum stage_)
|
||||
: query_ptr(query_ptr_), context(context_), stage(stage_)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
void InterpreterQuery::execute(WriteBuffer & ostr, ReadBuffer * remaining_data_istr, BlockInputStreamPtr & query_plan)
|
||||
{
|
||||
if (typeid_cast<ASTSelectQuery *>(&*query_ptr))
|
||||
{
|
||||
InterpreterSelectQuery interpreter(query_ptr, context, stage);
|
||||
query_plan = interpreter.executeAndFormat(ostr);
|
||||
}
|
||||
else if (typeid_cast<ASTInsertQuery *>(&*query_ptr))
|
||||
{
|
||||
throwIfReadOnly();
|
||||
InterpreterInsertQuery interpreter(query_ptr, context);
|
||||
interpreter.execute(remaining_data_istr);
|
||||
}
|
||||
else if (typeid_cast<ASTCreateQuery *>(&*query_ptr))
|
||||
{
|
||||
throwIfReadOnly();
|
||||
InterpreterCreateQuery interpreter(query_ptr, context);
|
||||
interpreter.execute();
|
||||
}
|
||||
else if (typeid_cast<ASTDropQuery *>(&*query_ptr))
|
||||
{
|
||||
throwIfReadOnly();
|
||||
InterpreterDropQuery interpreter(query_ptr, context);
|
||||
interpreter.execute();
|
||||
}
|
||||
else if (typeid_cast<ASTRenameQuery *>(&*query_ptr))
|
||||
{
|
||||
throwIfReadOnly();
|
||||
InterpreterRenameQuery interpreter(query_ptr, context);
|
||||
interpreter.execute();
|
||||
}
|
||||
else if (typeid_cast<ASTShowTablesQuery *>(&*query_ptr))
|
||||
{
|
||||
InterpreterShowTablesQuery interpreter(query_ptr, context);
|
||||
query_plan = interpreter.executeAndFormat(ostr);
|
||||
}
|
||||
else if (typeid_cast<ASTUseQuery *>(&*query_ptr))
|
||||
{
|
||||
InterpreterUseQuery interpreter(query_ptr, context);
|
||||
interpreter.execute();
|
||||
}
|
||||
else if (typeid_cast<ASTSetQuery *>(&*query_ptr))
|
||||
{
|
||||
/// readonly проверяется внутри InterpreterSetQuery
|
||||
InterpreterSetQuery interpreter(query_ptr, context);
|
||||
interpreter.execute();
|
||||
}
|
||||
else if (typeid_cast<ASTOptimizeQuery *>(&*query_ptr))
|
||||
{
|
||||
throwIfReadOnly();
|
||||
InterpreterOptimizeQuery interpreter(query_ptr, context);
|
||||
interpreter.execute();
|
||||
}
|
||||
else if (typeid_cast<ASTExistsQuery *>(&*query_ptr))
|
||||
{
|
||||
InterpreterExistsQuery interpreter(query_ptr, context);
|
||||
query_plan = interpreter.executeAndFormat(ostr);
|
||||
}
|
||||
else if (typeid_cast<ASTShowCreateQuery *>(&*query_ptr))
|
||||
{
|
||||
InterpreterShowCreateQuery interpreter(query_ptr, context);
|
||||
query_plan = interpreter.executeAndFormat(ostr);
|
||||
}
|
||||
else if (typeid_cast<ASTDescribeQuery *>(&*query_ptr))
|
||||
{
|
||||
InterpreterDescribeQuery interpreter(query_ptr, context);
|
||||
query_plan = interpreter.executeAndFormat(ostr);
|
||||
}
|
||||
else if (typeid_cast<ASTShowProcesslistQuery *>(&*query_ptr))
|
||||
{
|
||||
InterpreterShowProcesslistQuery interpreter(query_ptr, context);
|
||||
query_plan = interpreter.executeAndFormat(ostr);
|
||||
}
|
||||
else if (typeid_cast<ASTAlterQuery *>(&*query_ptr))
|
||||
{
|
||||
throwIfReadOnly();
|
||||
InterpreterAlterQuery interpreter(query_ptr, context);
|
||||
interpreter.execute();
|
||||
}
|
||||
else if (typeid_cast<ASTCheckQuery *>(&*query_ptr))
|
||||
{
|
||||
InterpreterCheckQuery interpreter(query_ptr, context);
|
||||
query_plan = interpreter.execute();
|
||||
}
|
||||
else
|
||||
throw Exception("Unknown type of query: " + query_ptr->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY);
|
||||
}
|
||||
|
||||
|
||||
BlockIO InterpreterQuery::execute()
|
||||
{
|
||||
BlockIO res;
|
||||
|
||||
if (typeid_cast<ASTSelectQuery *>(&*query_ptr))
|
||||
{
|
||||
InterpreterSelectQuery interpreter(query_ptr, context, stage);
|
||||
res.in = interpreter.execute();
|
||||
res.in_sample = interpreter.getSampleBlock();
|
||||
}
|
||||
else if (typeid_cast<ASTInsertQuery *>(&*query_ptr))
|
||||
{
|
||||
throwIfReadOnly();
|
||||
InterpreterInsertQuery interpreter(query_ptr, context);
|
||||
res = interpreter.execute();
|
||||
}
|
||||
else if (typeid_cast<ASTCreateQuery *>(&*query_ptr))
|
||||
{
|
||||
throwIfReadOnly();
|
||||
InterpreterCreateQuery interpreter(query_ptr, context);
|
||||
interpreter.execute();
|
||||
}
|
||||
else if (typeid_cast<ASTDropQuery *>(&*query_ptr))
|
||||
{
|
||||
throwIfReadOnly();
|
||||
InterpreterDropQuery interpreter(query_ptr, context);
|
||||
interpreter.execute();
|
||||
}
|
||||
else if (typeid_cast<ASTRenameQuery *>(&*query_ptr))
|
||||
{
|
||||
throwIfReadOnly();
|
||||
InterpreterRenameQuery interpreter(query_ptr, context);
|
||||
interpreter.execute();
|
||||
}
|
||||
else if (typeid_cast<ASTShowTablesQuery *>(&*query_ptr))
|
||||
{
|
||||
InterpreterShowTablesQuery interpreter(query_ptr, context);
|
||||
res = interpreter.execute();
|
||||
}
|
||||
else if (typeid_cast<ASTUseQuery *>(&*query_ptr))
|
||||
{
|
||||
InterpreterUseQuery interpreter(query_ptr, context);
|
||||
interpreter.execute();
|
||||
}
|
||||
else if (typeid_cast<ASTSetQuery *>(&*query_ptr))
|
||||
{
|
||||
/// readonly проверяется внутри InterpreterSetQuery
|
||||
InterpreterSetQuery interpreter(query_ptr, context);
|
||||
interpreter.execute();
|
||||
}
|
||||
else if (typeid_cast<ASTOptimizeQuery *>(&*query_ptr))
|
||||
{
|
||||
throwIfReadOnly();
|
||||
InterpreterOptimizeQuery interpreter(query_ptr, context);
|
||||
interpreter.execute();
|
||||
}
|
||||
else if (typeid_cast<ASTExistsQuery *>(&*query_ptr))
|
||||
{
|
||||
InterpreterExistsQuery interpreter(query_ptr, context);
|
||||
res = interpreter.execute();
|
||||
}
|
||||
else if (typeid_cast<ASTShowCreateQuery *>(&*query_ptr))
|
||||
{
|
||||
InterpreterShowCreateQuery interpreter(query_ptr, context);
|
||||
res = interpreter.execute();
|
||||
}
|
||||
else if (typeid_cast<ASTDescribeQuery *>(&*query_ptr))
|
||||
{
|
||||
InterpreterDescribeQuery interpreter(query_ptr, context);
|
||||
res = interpreter.execute();
|
||||
}
|
||||
else if (typeid_cast<ASTShowProcesslistQuery *>(&*query_ptr))
|
||||
{
|
||||
InterpreterShowProcesslistQuery interpreter(query_ptr, context);
|
||||
res = interpreter.execute();
|
||||
}
|
||||
else if (typeid_cast<ASTAlterQuery *>(&*query_ptr))
|
||||
{
|
||||
throwIfReadOnly();
|
||||
InterpreterAlterQuery interpreter(query_ptr, context);
|
||||
interpreter.execute();
|
||||
}
|
||||
else if (typeid_cast<ASTCheckQuery *>(&*query_ptr))
|
||||
{
|
||||
InterpreterCheckQuery interpreter(query_ptr, context);
|
||||
res.in = interpreter.execute();
|
||||
res.in_sample = interpreter.getSampleBlock();
|
||||
}
|
||||
else
|
||||
throw Exception("Unknown type of query: " + query_ptr->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
}
|
@ -58,7 +58,7 @@ struct RenameDescription
|
||||
};
|
||||
|
||||
|
||||
void InterpreterRenameQuery::execute()
|
||||
BlockIO InterpreterRenameQuery::execute()
|
||||
{
|
||||
String path = context.getPath();
|
||||
String current_database = context.getCurrentDatabase();
|
||||
@ -151,6 +151,8 @@ void InterpreterRenameQuery::execute()
|
||||
/// Удаляем старый файл с метаданными.
|
||||
Poco::File(elem.from_metadata_path).remove();
|
||||
}
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
|
||||
|
@ -297,12 +297,17 @@ Block InterpreterSelectQuery::getSampleBlock()
|
||||
}
|
||||
|
||||
|
||||
BlockInputStreamPtr InterpreterSelectQuery::execute()
|
||||
BlockIO InterpreterSelectQuery::execute()
|
||||
{
|
||||
(void) executeWithoutUnion();
|
||||
|
||||
if (streams.empty())
|
||||
return new NullBlockInputStream;
|
||||
{
|
||||
BlockIO res;
|
||||
res.in = new NullBlockInputStream;
|
||||
res.in_sample = getSampleBlock();
|
||||
return res;
|
||||
}
|
||||
|
||||
executeUnion(streams);
|
||||
|
||||
@ -326,7 +331,11 @@ BlockInputStreamPtr InterpreterSelectQuery::execute()
|
||||
}
|
||||
}
|
||||
|
||||
return streams[0];
|
||||
BlockIO res;
|
||||
res.in = streams[0];
|
||||
res.in_sample = getSampleBlock();
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
const BlockInputStreams & InterpreterSelectQuery::executeWithoutUnion()
|
||||
@ -1011,20 +1020,6 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(BlockInputStreams &
|
||||
}
|
||||
|
||||
|
||||
BlockInputStreamPtr InterpreterSelectQuery::executeAndFormat(WriteBuffer & buf)
|
||||
{
|
||||
Block sample = getSampleBlock();
|
||||
String format_name = query.format ? typeid_cast<ASTIdentifier &>(*query.format).name : context.getDefaultFormat();
|
||||
|
||||
BlockInputStreamPtr in = execute();
|
||||
BlockOutputStreamPtr out = context.getFormatFactory().getOutput(format_name, buf, sample);
|
||||
|
||||
copyData(*in, *out);
|
||||
|
||||
return in;
|
||||
}
|
||||
|
||||
|
||||
void InterpreterSelectQuery::ignoreWithTotals()
|
||||
{
|
||||
query.group_by_with_totals = false;
|
||||
|
@ -52,14 +52,4 @@ BlockIO InterpreterShowTablesQuery::execute()
|
||||
}
|
||||
|
||||
|
||||
BlockInputStreamPtr InterpreterShowTablesQuery::executeAndFormat(WriteBuffer & buf)
|
||||
{
|
||||
String query = getRewrittenQuery();
|
||||
ReadBufferFromString in(query);
|
||||
BlockInputStreamPtr query_plan;
|
||||
executeQuery(in, buf, context, query_plan, true);
|
||||
return query_plan;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
72
dbms/src/Interpreters/ProcessList.cpp
Normal file
72
dbms/src/Interpreters/ProcessList.cpp
Normal file
@ -0,0 +1,72 @@
|
||||
#include <DB/Interpreters/ProcessList.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
ProcessList::EntryPtr ProcessList::insert(
|
||||
const String & query_, const String & user_, const String & query_id_, const Poco::Net::IPAddress & ip_address_,
|
||||
size_t max_memory_usage, size_t max_wait_milliseconds, bool replace_running_query, QueryPriorities::Priority priority)
|
||||
{
|
||||
EntryPtr res;
|
||||
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
|
||||
if (max_size && cur_size >= max_size && (!max_wait_milliseconds || !have_space.tryWait(mutex, max_wait_milliseconds)))
|
||||
throw Exception("Too much simultaneous queries. Maximum: " + toString(max_size), ErrorCodes::TOO_MUCH_SIMULTANEOUS_QUERIES);
|
||||
|
||||
if (!query_id_.empty())
|
||||
{
|
||||
UserToQueries::iterator queries = user_to_queries.find(user_);
|
||||
|
||||
if (queries != user_to_queries.end())
|
||||
{
|
||||
QueryToElement::iterator element = queries->second.find(query_id_);
|
||||
if (element != queries->second.end())
|
||||
{
|
||||
if (!replace_running_query)
|
||||
throw Exception("Query with id = " + query_id_ + " is already running.",
|
||||
ErrorCodes::QUERY_WITH_SAME_ID_IS_ALREADY_RUNNING);
|
||||
element->second->is_cancelled = true;
|
||||
/// В случае если запрос отменяется, данные о нем удаляются из мапа в момент отмены.
|
||||
queries->second.erase(element);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
++cur_size;
|
||||
|
||||
res.reset(new Entry(*this, cont.emplace(cont.end(),
|
||||
query_, user_, query_id_, ip_address_, max_memory_usage, priorities.insert(priority))));
|
||||
|
||||
if (!query_id_.empty())
|
||||
user_to_queries[user_][query_id_] = &res->get();
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
ProcessListEntry::~ProcessListEntry()
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(parent.mutex);
|
||||
|
||||
/// В случае, если запрос отменяется, данные о нем удаляются из мапа в момент отмены.
|
||||
if (!it->is_cancelled && !it->query_id.empty())
|
||||
{
|
||||
ProcessList::UserToQueries::iterator queries = parent.user_to_queries.find(it->user);
|
||||
if (queries != parent.user_to_queries.end())
|
||||
{
|
||||
ProcessList::QueryToElement::iterator element = queries->second.find(it->query_id);
|
||||
if (element != queries->second.end())
|
||||
queries->second.erase(element);
|
||||
}
|
||||
}
|
||||
|
||||
parent.cont.erase(it);
|
||||
--parent.cur_size;
|
||||
parent.have_space.signal();
|
||||
}
|
||||
|
||||
}
|
279
dbms/src/Interpreters/QueryLog.cpp
Normal file
279
dbms/src/Interpreters/QueryLog.cpp
Normal file
@ -0,0 +1,279 @@
|
||||
#include <statdaemons/Stopwatch.h>
|
||||
#include <DB/Parsers/ASTCreateQuery.h>
|
||||
#include <DB/Parsers/parseQuery.h>
|
||||
#include <DB/Parsers/ExpressionElementParsers.h>
|
||||
#include <DB/Parsers/ASTRenameQuery.h>
|
||||
#include <DB/Parsers/formatAST.h>
|
||||
#include <DB/Columns/ColumnsNumber.h>
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
#include <DB/Columns/ColumnFixedString.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypeDateTime.h>
|
||||
#include <DB/DataTypes/DataTypeDate.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/DataTypes/DataTypeFixedString.h>
|
||||
#include <DB/Interpreters/InterpreterCreateQuery.h>
|
||||
#include <DB/Interpreters/InterpreterRenameQuery.h>
|
||||
#include <DB/Interpreters/QueryLog.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
QueryLog::QueryLog(Context & context_, const String & database_name_, const String & table_name_, size_t flush_interval_milliseconds_)
|
||||
: context(context_), database_name(database_name_), table_name(table_name_), flush_interval_milliseconds(flush_interval_milliseconds_)
|
||||
{
|
||||
data.reserve(DBMS_QUERY_LOG_QUEUE_SIZE);
|
||||
|
||||
{
|
||||
String description = backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name);
|
||||
|
||||
Poco::ScopedLock<Poco::Mutex> lock(context.getMutex());
|
||||
|
||||
table = context.tryGetTable(database_name, table_name);
|
||||
|
||||
if (table)
|
||||
{
|
||||
const Block expected = createBlock();
|
||||
const Block actual = table->getSampleBlockNonMaterialized();
|
||||
|
||||
if (!blocksHaveEqualStructure(actual, expected))
|
||||
{
|
||||
/// Переименовываем существующую таблицу.
|
||||
int suffix = 0;
|
||||
while (context.isTableExist(database_name, table_name + "_" + toString(suffix)))
|
||||
++suffix;
|
||||
|
||||
ASTRenameQuery * rename = new ASTRenameQuery;
|
||||
ASTPtr holder = rename;
|
||||
|
||||
ASTRenameQuery::Table from;
|
||||
from.database = database_name;
|
||||
from.table = table_name;
|
||||
|
||||
ASTRenameQuery::Table to;
|
||||
to.database = database_name;
|
||||
to.table = table_name + "_" + toString(suffix);
|
||||
|
||||
ASTRenameQuery::Element elem;
|
||||
elem.from = from;
|
||||
elem.to = to;
|
||||
|
||||
rename->elements.emplace_back(elem);
|
||||
|
||||
LOG_DEBUG(log, "Existing table " << description << " for query log has obsolete or different structure."
|
||||
" Renaming it to " << backQuoteIfNeed(to.table));
|
||||
|
||||
InterpreterRenameQuery(holder, context).execute();
|
||||
|
||||
/// Нужная таблица будет создана.
|
||||
table = nullptr;
|
||||
}
|
||||
else
|
||||
LOG_DEBUG(log, "Will use existing table " << description << " for query log.");
|
||||
}
|
||||
|
||||
if (!table)
|
||||
{
|
||||
/// Создаём таблицу.
|
||||
LOG_DEBUG(log, "Creating new table " << description << " for query log.");
|
||||
|
||||
ASTCreateQuery * create = new ASTCreateQuery;
|
||||
ASTPtr holder = create;
|
||||
|
||||
create->database = database_name;
|
||||
create->table = table_name;
|
||||
|
||||
Block sample = createBlock();
|
||||
create->columns = InterpreterCreateQuery::formatColumns(sample.getColumnsList());
|
||||
|
||||
String engine = "MergeTree(event_date, event_time, 8192)";
|
||||
ParserFunction engine_parser;
|
||||
|
||||
create->storage = parseQuery(engine_parser, engine.data(), engine.data() + engine.size(), "ENGINE to create table for query log");
|
||||
|
||||
InterpreterCreateQuery(holder, context).execute();
|
||||
|
||||
table = context.getTable(database_name, table_name);
|
||||
}
|
||||
}
|
||||
|
||||
saving_thread = std::thread([this] { threadFunction(); });
|
||||
}
|
||||
|
||||
|
||||
QueryLog::~QueryLog()
|
||||
{
|
||||
/// Говорим потоку, что надо завершиться.
|
||||
QueryLogElement elem;
|
||||
elem.type = QueryLogElement::SHUTDOWN;
|
||||
queue.push(elem);
|
||||
|
||||
saving_thread.join();
|
||||
}
|
||||
|
||||
|
||||
void QueryLog::threadFunction()
|
||||
{
|
||||
Stopwatch time_after_last_write;
|
||||
bool first = true;
|
||||
|
||||
while (true)
|
||||
{
|
||||
try
|
||||
{
|
||||
if (first)
|
||||
{
|
||||
time_after_last_write.restart();
|
||||
first = false;
|
||||
}
|
||||
|
||||
QueryLogElement element;
|
||||
bool has_element = false;
|
||||
|
||||
if (data.empty())
|
||||
{
|
||||
queue.pop(element);
|
||||
has_element = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t milliseconds_elapsed = time_after_last_write.elapsed() / 1000000;
|
||||
if (milliseconds_elapsed < flush_interval_milliseconds)
|
||||
has_element = queue.tryPop(element, flush_interval_milliseconds - milliseconds_elapsed);
|
||||
}
|
||||
|
||||
if (has_element)
|
||||
{
|
||||
if (element.type == QueryLogElement::SHUTDOWN)
|
||||
{
|
||||
flush();
|
||||
break;
|
||||
}
|
||||
else
|
||||
data.push_back(element);
|
||||
}
|
||||
|
||||
size_t milliseconds_elapsed = time_after_last_write.elapsed() / 1000000;
|
||||
if (milliseconds_elapsed >= flush_interval_milliseconds)
|
||||
{
|
||||
/// Записываем данные в таблицу.
|
||||
flush();
|
||||
time_after_last_write.restart();
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// В случае ошибки теряем накопленные записи, чтобы не блокироваться.
|
||||
data.clear();
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Block QueryLog::createBlock()
|
||||
{
|
||||
return
|
||||
{
|
||||
{new ColumnUInt8, new DataTypeUInt8, "type"},
|
||||
{new ColumnUInt16, new DataTypeDate, "event_date"},
|
||||
{new ColumnUInt32, new DataTypeDateTime, "event_time"},
|
||||
{new ColumnUInt32, new DataTypeDateTime, "query_start_time"},
|
||||
{new ColumnUInt64, new DataTypeUInt64, "query_duration_ms"},
|
||||
|
||||
{new ColumnUInt64, new DataTypeUInt64, "read_rows"},
|
||||
{new ColumnUInt64, new DataTypeUInt64, "read_bytes"},
|
||||
|
||||
{new ColumnUInt64, new DataTypeUInt64, "result_rows"},
|
||||
{new ColumnUInt64, new DataTypeUInt64, "result_bytes"},
|
||||
|
||||
{new ColumnUInt64, new DataTypeUInt64, "memory_usage"},
|
||||
|
||||
{new ColumnString, new DataTypeString, "query"},
|
||||
{new ColumnString, new DataTypeString, "exception"},
|
||||
{new ColumnString, new DataTypeString, "stack_trace"},
|
||||
|
||||
{new ColumnUInt8, new DataTypeUInt8, "interface"},
|
||||
{new ColumnUInt8, new DataTypeUInt8, "http_method"},
|
||||
{new ColumnFixedString(16), new DataTypeFixedString(16), "ip_address"},
|
||||
{new ColumnString, new DataTypeString, "user"},
|
||||
{new ColumnString, new DataTypeString, "query_id"},
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
void QueryLog::flush()
|
||||
{
|
||||
try
|
||||
{
|
||||
LOG_TRACE(log, "Flushing query log");
|
||||
|
||||
DateLUT & date_lut = DateLUT::instance();
|
||||
|
||||
Block block = createBlock();
|
||||
|
||||
for (const QueryLogElement & elem : data)
|
||||
{
|
||||
char ipv6_binary[16];
|
||||
if (Poco::Net::IPAddress::IPv6 == elem.ip_address.family())
|
||||
{
|
||||
memcpy(ipv6_binary, elem.ip_address.addr(), 16);
|
||||
}
|
||||
else if (Poco::Net::IPAddress::IPv4 == elem.ip_address.family())
|
||||
{
|
||||
/// Преобразуем в IPv6-mapped адрес.
|
||||
memset(ipv6_binary, 0, 10);
|
||||
ipv6_binary[10] = '\xFF';
|
||||
ipv6_binary[11] = '\xFF';
|
||||
memcpy(&ipv6_binary[12], elem.ip_address.addr(), 4);
|
||||
}
|
||||
else
|
||||
memset(ipv6_binary, 0, 16);
|
||||
|
||||
size_t i = 0;
|
||||
|
||||
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.type));
|
||||
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(date_lut.toDayNum(elem.event_time)));
|
||||
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.event_time));
|
||||
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.query_start_time));
|
||||
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.query_duration_ms));
|
||||
|
||||
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.read_rows));
|
||||
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.read_bytes));
|
||||
|
||||
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.result_rows));
|
||||
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.result_bytes));
|
||||
|
||||
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.memory_usage));
|
||||
|
||||
block.unsafeGetByPosition(i++).column.get()->insertData(elem.query.data(), elem.query.size());
|
||||
block.unsafeGetByPosition(i++).column.get()->insertData(elem.exception.data(), elem.exception.size());
|
||||
block.unsafeGetByPosition(i++).column.get()->insertData(elem.stack_trace.data(), elem.stack_trace.size());
|
||||
|
||||
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.interface));
|
||||
block.unsafeGetByPosition(i++).column.get()->insert(static_cast<UInt64>(elem.http_method));
|
||||
|
||||
block.unsafeGetByPosition(i++).column.get()->insertData(ipv6_binary, 16);
|
||||
|
||||
block.unsafeGetByPosition(i++).column.get()->insertData(elem.user.data(), elem.user.size());
|
||||
block.unsafeGetByPosition(i++).column.get()->insertData(elem.query_id.data(), elem.query_id.size());
|
||||
}
|
||||
|
||||
BlockOutputStreamPtr stream = table->write(nullptr);
|
||||
|
||||
stream->writePrefix();
|
||||
stream->write(block);
|
||||
stream->writeSuffix();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
|
||||
/// В случае ошибки тоже очищаем накопленные записи, чтобы не блокироваться.
|
||||
data.clear();
|
||||
}
|
||||
|
||||
}
|
@ -66,7 +66,7 @@ void QuotaForInterval::addQuery(time_t current_time, const String & quota_name)
|
||||
__sync_fetch_and_add(&used.queries, 1);
|
||||
}
|
||||
|
||||
void QuotaForInterval::addError(time_t current_time, const String & quota_name)
|
||||
void QuotaForInterval::addError(time_t current_time, const String & quota_name) noexcept
|
||||
{
|
||||
__sync_fetch_and_add(&used.errors, 1);
|
||||
}
|
||||
@ -181,7 +181,7 @@ void QuotaForIntervals::addQuery(time_t current_time)
|
||||
it->second.addQuery(current_time, name);
|
||||
}
|
||||
|
||||
void QuotaForIntervals::addError(time_t current_time)
|
||||
void QuotaForIntervals::addError(time_t current_time) noexcept
|
||||
{
|
||||
for (Container::reverse_iterator it = cont.rbegin(); it != cont.rend(); ++it)
|
||||
it->second.addError(current_time, name);
|
||||
|
@ -336,11 +336,37 @@ static Field convertToType(const Field & src, const IDataType & type)
|
||||
}
|
||||
else if (is_date || is_datetime)
|
||||
{
|
||||
if (src.getType() != Field::Types::UInt64)
|
||||
if (src.getType() == Field::Types::UInt64)
|
||||
return src;
|
||||
|
||||
if (src.getType() == Field::Types::String)
|
||||
{
|
||||
/// Возможность сравнивать даты и даты-с-временем со строкой.
|
||||
const String & str = src.get<const String &>();
|
||||
ReadBufferFromString in(str);
|
||||
|
||||
if (is_date)
|
||||
{
|
||||
DayNum_t date{};
|
||||
readDateText(date, in);
|
||||
if (!in.eof())
|
||||
throw Exception("String is too long for Date: " + str);
|
||||
|
||||
return Field(UInt64(date));
|
||||
}
|
||||
else
|
||||
{
|
||||
time_t date_time{};
|
||||
readDateTimeText(date_time, in);
|
||||
if (!in.eof())
|
||||
throw Exception("String is too long for DateTime: " + str);
|
||||
|
||||
return Field(UInt64(date_time));
|
||||
}
|
||||
}
|
||||
|
||||
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, "
|
||||
+ Field::Types::toString(src.getType()) + " at right");
|
||||
|
||||
return src;
|
||||
}
|
||||
}
|
||||
else
|
||||
|
@ -1,13 +1,23 @@
|
||||
#include <DB/Common/ProfileEvents.h>
|
||||
#include <DB/Common/formatReadable.h>
|
||||
|
||||
#include <DB/Parsers/formatAST.h>
|
||||
#include <DB/IO/ConcatReadBuffer.h>
|
||||
|
||||
#include <DB/DataStreams/BlockIO.h>
|
||||
#include <DB/DataStreams/FormatFactory.h>
|
||||
#include <DB/DataStreams/copyData.h>
|
||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||
|
||||
#include <DB/Parsers/ASTInsertQuery.h>
|
||||
#include <DB/Parsers/ASTShowProcesslistQuery.h>
|
||||
#include <DB/Parsers/ASTIdentifier.h>
|
||||
#include <DB/Parsers/ParserQuery.h>
|
||||
#include <DB/Parsers/parseQuery.h>
|
||||
|
||||
#include <DB/Interpreters/Quota.h>
|
||||
#include <DB/Interpreters/InterpreterFactory.h>
|
||||
#include <DB/Interpreters/ProcessList.h>
|
||||
#include <DB/Interpreters/QueryLog.h>
|
||||
#include <DB/Interpreters/executeQuery.h>
|
||||
|
||||
|
||||
@ -24,6 +34,277 @@ static void checkLimits(const IAST & ast, const Limits & limits)
|
||||
}
|
||||
|
||||
|
||||
/// Логгировать запрос в обычный лог (не в таблицу).
|
||||
static void logQuery(const String & query, const Context & context)
|
||||
{
|
||||
String logged_query = query;
|
||||
std::replace(logged_query.begin(), logged_query.end(), '\n', ' ');
|
||||
LOG_DEBUG(&Logger::get("executeQuery"), "(from " << context.getIPAddress().toString() << ") " << logged_query);
|
||||
}
|
||||
|
||||
|
||||
static void setClientInfo(QueryLogElement & elem, Context & context)
|
||||
{
|
||||
elem.interface = context.getInterface();
|
||||
elem.http_method = context.getHTTPMethod();
|
||||
elem.ip_address = context.getIPAddress();
|
||||
elem.user = context.getUser();
|
||||
elem.query_id = context.getCurrentQueryId();
|
||||
}
|
||||
|
||||
|
||||
static void onExceptionBeforeStart(const String & query, Context & context, time_t current_time)
|
||||
{
|
||||
/// Эксепшен до начала выполнения запроса.
|
||||
context.getQuota().addError(current_time);
|
||||
|
||||
bool log_queries = context.getSettingsRef().log_queries;
|
||||
|
||||
/// Логгируем в таблицу начало выполнения запроса, если нужно.
|
||||
if (log_queries)
|
||||
{
|
||||
QueryLogElement elem;
|
||||
|
||||
elem.type = QueryLogElement::EXCEPTION_BEFORE_START;
|
||||
|
||||
elem.event_time = current_time;
|
||||
elem.query_start_time = current_time;
|
||||
|
||||
elem.query = query;
|
||||
elem.exception = getCurrentExceptionMessage(false);
|
||||
|
||||
setClientInfo(elem, context);
|
||||
|
||||
try
|
||||
{
|
||||
throw;
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
elem.stack_trace = e.getStackTrace().toString();
|
||||
}
|
||||
catch (...) {}
|
||||
|
||||
context.getQueryLog().add(elem);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
IParser::Pos begin,
|
||||
IParser::Pos end,
|
||||
Context & context,
|
||||
bool internal,
|
||||
QueryProcessingStage::Enum stage)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::Query);
|
||||
time_t current_time = time(0);
|
||||
|
||||
ParserQuery parser;
|
||||
ASTPtr ast;
|
||||
size_t query_size;
|
||||
size_t max_query_size = context.getSettingsRef().max_query_size;
|
||||
|
||||
try
|
||||
{
|
||||
ast = parseQuery(parser, begin, end, "");
|
||||
|
||||
/// Засунем запрос в строку. Она выводится в лог и в processlist. Если запрос INSERT, то не будем включать данные для вставки.
|
||||
query_size = ast->range.second - ast->range.first;
|
||||
|
||||
if (max_query_size && query_size > max_query_size)
|
||||
throw Exception("Query is too large (" + toString(query_size) + ")."
|
||||
" max_query_size = " + toString(max_query_size), ErrorCodes::QUERY_IS_TOO_LARGE);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Всё равно логгируем запрос.
|
||||
if (!internal)
|
||||
{
|
||||
String query = String(begin, begin + std::min(end - begin, static_cast<ptrdiff_t>(max_query_size)));
|
||||
logQuery(query, context);
|
||||
onExceptionBeforeStart(query, context, current_time);
|
||||
}
|
||||
|
||||
throw;
|
||||
}
|
||||
|
||||
String query(begin, query_size);
|
||||
BlockIO res;
|
||||
|
||||
try
|
||||
{
|
||||
if (!internal)
|
||||
logQuery(query, context);
|
||||
|
||||
/// Проверка ограничений.
|
||||
checkLimits(*ast, context.getSettingsRef().limits);
|
||||
|
||||
QuotaForIntervals & quota = context.getQuota();
|
||||
|
||||
quota.checkExceeded(current_time);
|
||||
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
|
||||
/// Положим запрос в список процессов. Но запрос SHOW PROCESSLIST класть не будем.
|
||||
ProcessList::EntryPtr process_list_entry;
|
||||
if (!internal && nullptr == typeid_cast<const ASTShowProcesslistQuery *>(&*ast))
|
||||
{
|
||||
process_list_entry = context.getProcessList().insert(
|
||||
query, context.getUser(), context.getCurrentQueryId(), context.getIPAddress(),
|
||||
settings.limits.max_memory_usage,
|
||||
settings.queue_max_wait_ms.totalMilliseconds(),
|
||||
settings.replace_running_query,
|
||||
settings.priority);
|
||||
|
||||
context.setProcessListElement(&process_list_entry->get());
|
||||
}
|
||||
|
||||
auto interpreter = InterpreterFactory::get(ast, context, stage);
|
||||
res = interpreter->execute();
|
||||
|
||||
/// Держим элемент списка процессов до конца обработки запроса.
|
||||
res.process_list_entry = process_list_entry;
|
||||
|
||||
quota.addQuery(current_time);
|
||||
|
||||
/// Всё, что связано с логом запросов.
|
||||
{
|
||||
QueryLogElement elem;
|
||||
|
||||
elem.type = QueryLogElement::QUERY_START;
|
||||
|
||||
elem.event_time = current_time;
|
||||
elem.query_start_time = current_time;
|
||||
|
||||
elem.query = query;
|
||||
|
||||
setClientInfo(elem, context);
|
||||
|
||||
bool log_queries = settings.log_queries;
|
||||
|
||||
/// Логгируем в таблицу начало выполнения запроса, если нужно.
|
||||
if (log_queries)
|
||||
context.getQueryLog().add(elem);
|
||||
|
||||
/// Также дадим вызывающему коду в дальнейшем логгировать завершение запроса и эксепшен.
|
||||
res.finish_callback = [elem, &context, log_queries] (IBlockInputStream & stream) mutable
|
||||
{
|
||||
ProcessListElement * process_list_elem = context.getProcessListElement();
|
||||
|
||||
if (!process_list_elem)
|
||||
return;
|
||||
|
||||
double elapsed_seconds = process_list_elem->watch.elapsedSeconds();
|
||||
|
||||
elem.type = QueryLogElement::QUERY_FINISH;
|
||||
|
||||
elem.event_time = time(0);
|
||||
elem.query_duration_ms = elapsed_seconds * 1000;
|
||||
|
||||
elem.read_rows = process_list_elem->progress.rows;
|
||||
elem.read_bytes = process_list_elem->progress.bytes;
|
||||
|
||||
auto memory_usage = process_list_elem->memory_tracker.getPeak();
|
||||
elem.memory_usage = memory_usage > 0 ? memory_usage : 0;
|
||||
|
||||
if (IProfilingBlockInputStream * profiling_stream = dynamic_cast<IProfilingBlockInputStream *>(&stream))
|
||||
{
|
||||
const BlockStreamProfileInfo & info = profiling_stream->getInfo();
|
||||
|
||||
elem.result_rows = info.rows;
|
||||
elem.result_bytes = info.bytes;
|
||||
}
|
||||
|
||||
if (elem.read_rows != 0)
|
||||
{
|
||||
LOG_INFO(&Logger::get("executeQuery"), std::fixed << std::setprecision(3)
|
||||
<< "Read " << elem.read_rows << " rows, "
|
||||
<< formatReadableSizeWithBinarySuffix(elem.read_bytes) << " in " << elapsed_seconds << " sec., "
|
||||
<< static_cast<size_t>(elem.read_rows / elapsed_seconds) << " rows/sec., "
|
||||
<< formatReadableSizeWithBinarySuffix(elem.read_bytes / elapsed_seconds) << "/sec.");
|
||||
}
|
||||
|
||||
if (log_queries)
|
||||
context.getQueryLog().add(elem);
|
||||
};
|
||||
|
||||
res.exception_callback = [elem, &context, log_queries, current_time] () mutable
|
||||
{
|
||||
context.getQuota().addError(current_time);
|
||||
|
||||
elem.type = QueryLogElement::EXCEPTION_WHILE_PROCESSING;
|
||||
|
||||
elem.event_time = time(0);
|
||||
elem.query_duration_ms = 1000 * (elem.event_time - elem.query_start_time);
|
||||
elem.exception = getCurrentExceptionMessage(false);
|
||||
|
||||
ProcessListElement * process_list_elem = context.getProcessListElement();
|
||||
|
||||
if (process_list_elem)
|
||||
{
|
||||
double elapsed_seconds = process_list_elem->watch.elapsedSeconds();
|
||||
|
||||
elem.query_duration_ms = elapsed_seconds * 1000;
|
||||
|
||||
elem.read_rows = process_list_elem->progress.rows;
|
||||
elem.read_bytes = process_list_elem->progress.bytes;
|
||||
|
||||
auto memory_usage = process_list_elem->memory_tracker.getPeak();
|
||||
elem.memory_usage = memory_usage > 0 ? memory_usage : 0;
|
||||
}
|
||||
|
||||
/// Достаём стек трейс, если возможно.
|
||||
try
|
||||
{
|
||||
throw;
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
elem.stack_trace = e.getStackTrace().toString();
|
||||
|
||||
LOG_ERROR(&Logger::get("executeQuery"), elem.exception << ", Stack trace:\n\n" << elem.stack_trace);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LOG_ERROR(&Logger::get("executeQuery"), elem.exception);
|
||||
}
|
||||
|
||||
if (log_queries)
|
||||
context.getQueryLog().add(elem);
|
||||
};
|
||||
|
||||
if (!internal && res.in)
|
||||
{
|
||||
std::stringstream log_str;
|
||||
log_str << "Query pipeline:\n";
|
||||
res.in->dumpTree(log_str);
|
||||
LOG_DEBUG(&Logger::get("executeQuery"), log_str.str());
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
onExceptionBeforeStart(query, context, current_time);
|
||||
throw;
|
||||
}
|
||||
|
||||
return std::make_tuple(ast, res);
|
||||
}
|
||||
|
||||
|
||||
BlockIO executeQuery(
|
||||
const String & query,
|
||||
Context & context,
|
||||
bool internal,
|
||||
QueryProcessingStage::Enum stage)
|
||||
{
|
||||
BlockIO streams;
|
||||
std::tie(std::ignore, streams) = executeQueryImpl(query.data(), query.data() + query.size(), context, internal, stage);
|
||||
return streams;
|
||||
}
|
||||
|
||||
|
||||
void executeQuery(
|
||||
ReadBuffer & istr,
|
||||
WriteBuffer & ostr,
|
||||
@ -32,10 +313,6 @@ void executeQuery(
|
||||
bool internal,
|
||||
QueryProcessingStage::Enum stage)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::Query);
|
||||
|
||||
ParserQuery parser;
|
||||
|
||||
PODArray<char> parse_buf;
|
||||
const char * begin;
|
||||
const char * end;
|
||||
@ -44,7 +321,7 @@ void executeQuery(
|
||||
if (istr.buffer().size() == 0)
|
||||
istr.next();
|
||||
|
||||
size_t max_query_size = context.getSettings().max_query_size;
|
||||
size_t max_query_size = context.getSettingsRef().max_query_size;
|
||||
|
||||
if (istr.buffer().end() - istr.position() >= static_cast<ssize_t>(max_query_size))
|
||||
{
|
||||
@ -62,108 +339,66 @@ void executeQuery(
|
||||
end = begin + parse_buf.size();
|
||||
}
|
||||
|
||||
ASTPtr ast = parseQuery(parser, begin, end, "");
|
||||
ASTPtr ast;
|
||||
BlockIO streams;
|
||||
|
||||
/// Засунем запрос в строку. Она выводится в лог и в processlist. Если запрос INSERT, то не будем включать данные для вставки.
|
||||
size_t query_size = ast->range.second - ast->range.first;
|
||||
|
||||
if (query_size > max_query_size)
|
||||
throw Exception("Query is too large (" + toString(query_size) + ")."
|
||||
" max_query_size = " + toString(max_query_size), ErrorCodes::QUERY_IS_TOO_LARGE);
|
||||
|
||||
String query(begin, query_size);
|
||||
|
||||
String logged_query = query;
|
||||
std::replace(logged_query.begin(), logged_query.end(), '\n', ' ');
|
||||
LOG_DEBUG(&Logger::get("executeQuery"), logged_query);
|
||||
|
||||
/// Положим запрос в список процессов. Но запрос SHOW PROCESSLIST класть не будем.
|
||||
ProcessList::EntryPtr process_list_entry;
|
||||
if (!internal && nullptr == typeid_cast<const ASTShowProcesslistQuery *>(&*ast))
|
||||
{
|
||||
process_list_entry = context.getProcessList().insert(
|
||||
query, context.getUser(), context.getCurrentQueryId(), context.getIPAddress(),
|
||||
context.getSettingsRef().limits.max_memory_usage,
|
||||
context.getSettingsRef().queue_max_wait_ms.totalMilliseconds(),
|
||||
context.getSettingsRef().replace_running_query);
|
||||
|
||||
context.setProcessListElement(&process_list_entry->get());
|
||||
}
|
||||
|
||||
/// Проверка ограничений.
|
||||
checkLimits(*ast, context.getSettingsRef().limits);
|
||||
|
||||
QuotaForIntervals & quota = context.getQuota();
|
||||
time_t current_time = time(0);
|
||||
|
||||
quota.checkExceeded(current_time);
|
||||
std::tie(ast, streams) = executeQueryImpl(begin, end, context, internal, stage);
|
||||
|
||||
try
|
||||
{
|
||||
InterpreterQuery interpreter(ast, context, stage);
|
||||
interpreter.execute(ostr, &istr, query_plan);
|
||||
if (streams.out)
|
||||
{
|
||||
const ASTInsertQuery * ast_insert_query = dynamic_cast<const ASTInsertQuery *>(ast.get());
|
||||
|
||||
if (!ast_insert_query)
|
||||
throw Exception("Logical error: query requires data to insert, but it is not INSERT query", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
String format = ast_insert_query->format;
|
||||
if (format.empty())
|
||||
format = "Values";
|
||||
|
||||
/// Данные могут содержаться в распарсенной (ast_insert_query.data) и ещё не распарсенной (istr) части запроса.
|
||||
|
||||
ConcatReadBuffer::ReadBuffers buffers;
|
||||
ReadBuffer buf1(const_cast<char *>(ast_insert_query->data), ast_insert_query->data ? ast_insert_query->end - ast_insert_query->data : 0, 0);
|
||||
|
||||
if (ast_insert_query->data)
|
||||
buffers.push_back(&buf1);
|
||||
buffers.push_back(&istr);
|
||||
|
||||
/** NOTE Нельзя читать из istr до того, как прочтём всё между ast_insert_query.data и ast_insert_query.end.
|
||||
* - потому что query.data может ссылаться на кусок памяти, использующийся в качестве буфера в istr.
|
||||
*/
|
||||
|
||||
ConcatReadBuffer data_istr(buffers);
|
||||
|
||||
BlockInputStreamPtr in{
|
||||
context.getFormatFactory().getInput(
|
||||
format, data_istr, streams.out_sample, context.getSettings().max_insert_block_size)};
|
||||
|
||||
copyData(*in, *streams.out);
|
||||
}
|
||||
|
||||
if (streams.in)
|
||||
{
|
||||
const ASTQueryWithOutput * ast_query_with_output = dynamic_cast<const ASTQueryWithOutput *>(ast.get());
|
||||
|
||||
String format_name = ast_query_with_output && (ast_query_with_output->getFormat() != nullptr)
|
||||
? typeid_cast<const ASTIdentifier &>(*ast_query_with_output->getFormat()).name
|
||||
: context.getDefaultFormat();
|
||||
|
||||
BlockOutputStreamPtr out = context.getFormatFactory().getOutput(format_name, ostr, streams.in_sample);
|
||||
|
||||
copyData(*streams.in, *out);
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
quota.addError(current_time);
|
||||
streams.onException();
|
||||
throw;
|
||||
}
|
||||
|
||||
quota.addQuery(current_time);
|
||||
streams.onFinish();
|
||||
}
|
||||
|
||||
|
||||
BlockIO executeQuery(
|
||||
const String & query,
|
||||
Context & context,
|
||||
bool internal,
|
||||
QueryProcessingStage::Enum stage)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::Query);
|
||||
|
||||
ParserQuery parser;
|
||||
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "");
|
||||
|
||||
/// Проверка ограничений.
|
||||
checkLimits(*ast, context.getSettingsRef().limits);
|
||||
|
||||
QuotaForIntervals & quota = context.getQuota();
|
||||
time_t current_time = time(0);
|
||||
|
||||
quota.checkExceeded(current_time);
|
||||
|
||||
BlockIO res;
|
||||
|
||||
/// Положим запрос в список процессов. Но запрос SHOW PROCESSLIST класть не будем.
|
||||
ProcessList::EntryPtr process_list_entry;
|
||||
if (!internal && nullptr == typeid_cast<const ASTShowProcesslistQuery *>(&*ast))
|
||||
{
|
||||
process_list_entry = context.getProcessList().insert(
|
||||
query, context.getUser(), context.getCurrentQueryId(), context.getIPAddress(),
|
||||
context.getSettingsRef().limits.max_memory_usage,
|
||||
context.getSettingsRef().queue_max_wait_ms.totalMilliseconds(),
|
||||
context.getSettingsRef().replace_running_query);
|
||||
|
||||
context.setProcessListElement(&process_list_entry->get());
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
InterpreterQuery interpreter(ast, context, stage);
|
||||
res = interpreter.execute();
|
||||
|
||||
/// Держим элемент списка процессов до конца обработки запроса.
|
||||
res.process_list_entry = process_list_entry;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
quota.addError(current_time);
|
||||
throw;
|
||||
}
|
||||
|
||||
quota.addQuery(current_time);
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -30,8 +30,7 @@ static void executeCreateQuery(const String & query, Context & context, const St
|
||||
ast_create_query.attach = true;
|
||||
ast_create_query.database = database;
|
||||
|
||||
InterpreterCreateQuery interpreter(ast, context);
|
||||
interpreter.execute(true);
|
||||
InterpreterCreateQuery(ast, context).executeLoadExisting();
|
||||
}
|
||||
|
||||
|
||||
|
82
dbms/src/Parsers/ASTLiteral.cpp
Normal file
82
dbms/src/Parsers/ASTLiteral.cpp
Normal file
@ -0,0 +1,82 @@
|
||||
#include <DB/Common/SipHash.h>
|
||||
#include <DB/Parsers/ASTLiteral.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
/** Обновляет SipHash по данным Field */
|
||||
class FieldVisitorHash : public StaticVisitor<>
|
||||
{
|
||||
private:
|
||||
SipHash & hash;
|
||||
public:
|
||||
FieldVisitorHash(SipHash & hash) : hash(hash) {}
|
||||
|
||||
void operator() (const Null & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Null;
|
||||
hash.update(reinterpret_cast<const char *>(&type), sizeof(type));
|
||||
}
|
||||
|
||||
void operator() (const UInt64 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::UInt64;
|
||||
hash.update(reinterpret_cast<const char *>(&type), sizeof(type));
|
||||
hash.update(reinterpret_cast<const char *>(&x), sizeof(x));
|
||||
}
|
||||
|
||||
void operator() (const Int64 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Int64;
|
||||
hash.update(reinterpret_cast<const char *>(&type), sizeof(type));
|
||||
hash.update(reinterpret_cast<const char *>(&x), sizeof(x));
|
||||
}
|
||||
|
||||
void operator() (const Float64 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Float64;
|
||||
hash.update(reinterpret_cast<const char *>(&type), sizeof(type));
|
||||
hash.update(reinterpret_cast<const char *>(&x), sizeof(x));
|
||||
}
|
||||
|
||||
void operator() (const String & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::String;
|
||||
hash.update(reinterpret_cast<const char *>(&type), sizeof(type));
|
||||
size_t size = x.size();
|
||||
hash.update(reinterpret_cast<const char *>(&size), sizeof(size));
|
||||
hash.update(x.data(), x.size());
|
||||
}
|
||||
|
||||
void operator() (const Array & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Array;
|
||||
hash.update(reinterpret_cast<const char *>(&type), sizeof(type));
|
||||
size_t size = x.size();
|
||||
hash.update(reinterpret_cast<const char *>(&size), sizeof(size));
|
||||
|
||||
for (const auto & elem : x)
|
||||
apply_visitor(*this, elem);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
String ASTLiteral::getColumnName() const
|
||||
{
|
||||
/// Отдельный случай для очень больших массивов. Вместо указания всех элементов, будем использовать хэш от содержимого.
|
||||
if (value.getType() == Field::Types::Array
|
||||
&& value.get<const Array &>().size() > 100) /// 100 - наугад.
|
||||
{
|
||||
SipHash hash;
|
||||
apply_visitor(FieldVisitorHash(hash), value);
|
||||
UInt64 low, high;
|
||||
hash.get128(low, high);
|
||||
return "__array_" + toString(low) + "_" + toString(high);
|
||||
}
|
||||
|
||||
return apply_visitor(FieldVisitorToString(), value);
|
||||
}
|
||||
|
||||
}
|
@ -172,5 +172,14 @@ ASTPtr ASTSelectQuery::clone() const
|
||||
|
||||
return ptr;
|
||||
}
|
||||
|
||||
const IAST * ASTSelectQuery::getFormat() const
|
||||
{
|
||||
const ASTSelectQuery * query = this;
|
||||
while (!query->next_union_all.isNull())
|
||||
query = static_cast<const ASTSelectQuery *>(query->next_union_all.get());
|
||||
return query->format.get();
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
|
@ -405,6 +405,59 @@ bool ParserStringLiteral::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max
|
||||
}
|
||||
|
||||
|
||||
bool ParserArrayOfLiterals::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
|
||||
{
|
||||
Pos begin = pos;
|
||||
Array arr;
|
||||
|
||||
if (pos == end || *pos != '[')
|
||||
{
|
||||
expected = "opening square bracket";
|
||||
return false;
|
||||
}
|
||||
|
||||
ParserWhiteSpaceOrComments ws;
|
||||
ParserLiteral literal_p;
|
||||
|
||||
++pos;
|
||||
|
||||
while (pos != end)
|
||||
{
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (!arr.empty())
|
||||
{
|
||||
if (*pos == ']')
|
||||
{
|
||||
++pos;
|
||||
node = new ASTLiteral(StringRange(begin, pos), arr);
|
||||
return true;
|
||||
}
|
||||
else if (*pos == ',')
|
||||
{
|
||||
++pos;
|
||||
}
|
||||
else
|
||||
{
|
||||
expected = "comma or closing square bracket";
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
ASTPtr literal_node;
|
||||
if (!literal_p.parse(pos, end, literal_node, max_parsed_pos, expected))
|
||||
return false;
|
||||
|
||||
arr.push_back(typeid_cast<const ASTLiteral &>(*literal_node).value);
|
||||
}
|
||||
|
||||
expected = "closing square bracket";
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
bool ParserLiteral::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
|
||||
{
|
||||
ParserNull null_p;
|
||||
@ -450,6 +503,7 @@ bool ParserExpressionElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos &
|
||||
ParserParenthesisExpression paren_p;
|
||||
ParserSubquery subquery_p;
|
||||
ParserArray array_p;
|
||||
ParserArrayOfLiterals array_lite_p;
|
||||
ParserLiteral lit_p;
|
||||
ParserFunction fun_p;
|
||||
ParserCompoundIdentifier id_p;
|
||||
@ -461,6 +515,9 @@ bool ParserExpressionElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos &
|
||||
if (paren_p.parse(pos, end, node, max_parsed_pos, expected))
|
||||
return true;
|
||||
|
||||
if (array_lite_p.parse(pos, end, node, max_parsed_pos, expected))
|
||||
return true;
|
||||
|
||||
if (array_p.parse(pos, end, node, max_parsed_pos, expected))
|
||||
return true;
|
||||
|
||||
|
@ -150,6 +150,12 @@ bool ParserAlterQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_pa
|
||||
{
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (s_unreplicated.ignore(pos, end, max_parsed_pos, expected))
|
||||
{
|
||||
params.unreplicated = true;
|
||||
ws.ignore(pos, end);
|
||||
}
|
||||
|
||||
if (!s_partition.ignore(pos, end, max_parsed_pos, expected))
|
||||
return false;
|
||||
|
||||
|
@ -8,7 +8,6 @@
|
||||
#include <DB/Parsers/ParserSetQuery.h>
|
||||
#include <DB/Parsers/ParserSelectQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -296,6 +295,8 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
ws.ignore(pos, end);
|
||||
}
|
||||
|
||||
bool has_format = false;
|
||||
|
||||
/// FORMAT format_name
|
||||
if (s_format.ignore(pos, end, max_parsed_pos, expected))
|
||||
{
|
||||
@ -308,6 +309,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
typeid_cast<ASTIdentifier &>(*select_query->format).kind = ASTIdentifier::Format;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
has_format = true;
|
||||
}
|
||||
|
||||
// UNION ALL select query
|
||||
@ -317,6 +319,13 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
|
||||
if (s_all.ignore(pos, end, max_parsed_pos, expected))
|
||||
{
|
||||
if (has_format)
|
||||
{
|
||||
/// FORMAT может быть задан только в последнем запросе цепочки UNION ALL.
|
||||
expected = "FORMAT only in the last SELECT of the UNION ALL chain";
|
||||
return false;
|
||||
}
|
||||
|
||||
ParserSelectQuery select_p;
|
||||
if (!select_p.parse(pos, end, select_query->next_union_all, max_parsed_pos, expected))
|
||||
return false;
|
||||
|
@ -136,41 +136,24 @@ void HTTPHandler::processQuery(Poco::Net::HTTPServerRequest & request, Poco::Net
|
||||
if (readonly)
|
||||
context.getSettingsRef().limits.readonly = true;
|
||||
|
||||
Stopwatch watch;
|
||||
context.setInterface(Context::Interface::HTTP);
|
||||
|
||||
Context::HTTPMethod http_method = Context::HTTPMethod::UNKNOWN;
|
||||
if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET)
|
||||
http_method = Context::HTTPMethod::GET;
|
||||
else if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_POST)
|
||||
http_method = Context::HTTPMethod::POST;
|
||||
|
||||
context.setHTTPMethod(http_method);
|
||||
|
||||
executeQuery(*in, *used_output.out_maybe_compressed, context, query_plan);
|
||||
watch.stop();
|
||||
|
||||
if (query_plan)
|
||||
{
|
||||
std::stringstream log_str;
|
||||
log_str << "Query pipeline:\n";
|
||||
query_plan->dumpTree(log_str);
|
||||
LOG_DEBUG(log, log_str.str());
|
||||
|
||||
/// Выведем информацию о том, сколько считано строк и байт.
|
||||
size_t rows = 0;
|
||||
size_t bytes = 0;
|
||||
|
||||
query_plan->getLeafRowsBytes(rows, bytes);
|
||||
|
||||
if (rows != 0)
|
||||
{
|
||||
LOG_INFO(log, std::fixed << std::setprecision(3)
|
||||
<< "Read " << rows << " rows, " << bytes / 1048576.0 << " MiB in " << watch.elapsedSeconds() << " sec., "
|
||||
<< static_cast<size_t>(rows / watch.elapsedSeconds()) << " rows/sec., " << bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.");
|
||||
}
|
||||
}
|
||||
|
||||
QuotaForIntervals & quota = context.getQuota();
|
||||
if (!quota.empty())
|
||||
LOG_INFO(log, "Quota:\n" << quota.toString());
|
||||
|
||||
/// Если не было эксепшена и данные ещё не отправлены - отправляются HTTP заголовки с кодом 200.
|
||||
used_output.out->finalize();
|
||||
}
|
||||
|
||||
|
||||
void HTTPHandler::trySendExceptionToClient(std::stringstream & s,
|
||||
void HTTPHandler::trySendExceptionToClient(const std::string & s,
|
||||
Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response,
|
||||
Output & used_output)
|
||||
{
|
||||
@ -191,7 +174,7 @@ void HTTPHandler::trySendExceptionToClient(std::stringstream & s,
|
||||
if (!response.sent() && !used_output.out_maybe_compressed)
|
||||
{
|
||||
/// Ещё ничего не отправляли, и даже не знаем, нужно ли сжимать ответ.
|
||||
response.send() << s.str() << std::endl;
|
||||
response.send() << s << std::endl;
|
||||
}
|
||||
else if (used_output.out_maybe_compressed)
|
||||
{
|
||||
@ -207,8 +190,7 @@ void HTTPHandler::trySendExceptionToClient(std::stringstream & s,
|
||||
used_output.out->position() = used_output.out->buffer().begin();
|
||||
}
|
||||
|
||||
std::string exception_message = s.str();
|
||||
writeString(exception_message, *used_output.out_maybe_compressed);
|
||||
writeString(s, *used_output.out_maybe_compressed);
|
||||
writeChar('\n', *used_output.out_maybe_compressed);
|
||||
used_output.out_maybe_compressed->next();
|
||||
used_output.out->finalize();
|
||||
@ -245,35 +227,10 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
|
||||
processQuery(request, response, used_output);
|
||||
LOG_INFO(log, "Done processing query");
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
std::stringstream s;
|
||||
s << "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what()
|
||||
<< ", Stack trace:\n\n" << e.getStackTrace().toString();
|
||||
LOG_ERROR(log, s.str());
|
||||
trySendExceptionToClient(s, request, response, used_output);
|
||||
}
|
||||
catch (Poco::Exception & e)
|
||||
{
|
||||
std::stringstream s;
|
||||
s << "Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
|
||||
<< ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what();
|
||||
LOG_ERROR(log, s.str());
|
||||
trySendExceptionToClient(s, request, response, used_output);
|
||||
}
|
||||
catch (std::exception & e)
|
||||
{
|
||||
std::stringstream s;
|
||||
s << "Code: " << ErrorCodes::STD_EXCEPTION << ". " << e.what();
|
||||
LOG_ERROR(log, s.str());
|
||||
trySendExceptionToClient(s, request, response, used_output);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
std::stringstream s;
|
||||
s << "Code: " << ErrorCodes::UNKNOWN_EXCEPTION << ". Unknown exception.";
|
||||
LOG_ERROR(log, s.str());
|
||||
trySendExceptionToClient(s, request, response, used_output);
|
||||
tryLogCurrentException(log);
|
||||
trySendExceptionToClient(getCurrentExceptionMessage(true), request, response, used_output);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -26,7 +26,7 @@ public:
|
||||
|
||||
void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response);
|
||||
|
||||
void trySendExceptionToClient(std::stringstream & s,
|
||||
void trySendExceptionToClient(const std::string & s,
|
||||
Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response,
|
||||
Output & used_output);
|
||||
|
||||
|
@ -52,6 +52,9 @@ void OLAPHTTPHandler::processQuery(Poco::Net::HTTPServerRequest & request, Poco:
|
||||
|
||||
context.setUser(user, password, request.clientAddress().host(), quota_key);
|
||||
|
||||
context.setInterface(Context::Interface::HTTP);
|
||||
context.setHTTPMethod(Context::HTTPMethod::POST);
|
||||
|
||||
OLAP::QueryParseResult olap_query = server.olap_parser->parse(request_istream);
|
||||
|
||||
std::string clickhouse_query;
|
||||
@ -66,31 +69,6 @@ void OLAPHTTPHandler::processQuery(Poco::Net::HTTPServerRequest & request, Poco:
|
||||
executeQuery(in, out, context, query_plan);
|
||||
watch.stop();
|
||||
|
||||
if (query_plan)
|
||||
{
|
||||
std::stringstream log_str;
|
||||
log_str << "Query pipeline:\n";
|
||||
query_plan->dumpTree(log_str);
|
||||
LOG_DEBUG(log, log_str.str());
|
||||
|
||||
/// Выведем информацию о том, сколько считано строк и байт.
|
||||
size_t rows = 0;
|
||||
size_t bytes = 0;
|
||||
|
||||
query_plan->getLeafRowsBytes(rows, bytes);
|
||||
|
||||
if (rows != 0)
|
||||
{
|
||||
LOG_INFO(log, std::fixed << std::setprecision(3)
|
||||
<< "Read " << rows << " rows, " << bytes / 1048576.0 << " MiB in " << watch.elapsedSeconds() << " sec., "
|
||||
<< static_cast<size_t>(rows / watch.elapsedSeconds()) << " rows/sec., " << bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.");
|
||||
}
|
||||
}
|
||||
|
||||
QuotaForIntervals & quota = context.getQuota();
|
||||
if (!quota.empty())
|
||||
LOG_INFO(log, "Quota:\n" << quota.toString());
|
||||
|
||||
/// Если не было эксепшена и данные ещё не отправлены - отправляются HTTP заголовки с кодом 200.
|
||||
out.finalize();
|
||||
}
|
||||
|
@ -20,6 +20,7 @@
|
||||
#include <DB/Common/Macros.h>
|
||||
#include <DB/Common/getFQDNOrHostName.h>
|
||||
#include <DB/Interpreters/loadMetadata.h>
|
||||
#include <DB/Interpreters/ProcessList.h>
|
||||
#include <DB/Storages/StorageSystemNumbers.h>
|
||||
#include <DB/Storages/StorageSystemTables.h>
|
||||
#include <DB/Storages/StorageSystemParts.h>
|
||||
@ -532,7 +533,12 @@ int Server::main(const std::vector<std::string> & args)
|
||||
LOG_DEBUG(log, "Loaded metadata.");
|
||||
|
||||
/// Создаём системные таблицы.
|
||||
if (!global_context->isDatabaseExist("system"))
|
||||
{
|
||||
Poco::File(path + "data/system").createDirectories();
|
||||
Poco::File(path + "metadata/system").createDirectories();
|
||||
global_context->addDatabase("system");
|
||||
}
|
||||
|
||||
global_context->addTable("system", "one", StorageSystemOne::create("one"));
|
||||
global_context->addTable("system", "numbers", StorageSystemNumbers::create("numbers"));
|
||||
|
@ -111,6 +111,7 @@ void TCPHandler::runImpl()
|
||||
{
|
||||
/// Восстанавливаем контекст запроса.
|
||||
query_context = connection_context;
|
||||
query_context.setInterface(Context::Interface::TCP);
|
||||
|
||||
/** Если Query - обрабатываем. Если Ping или Cancel - возвращаемся в начало.
|
||||
* Могут прийти настройки на отдельный запрос, которые модифицируют query_context.
|
||||
@ -148,8 +149,7 @@ void TCPHandler::runImpl()
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
LOG_ERROR(log, "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what()
|
||||
<< ", Stack trace:\n\n" << e.getStackTrace().toString());
|
||||
state.io.onException();
|
||||
exception = e.clone();
|
||||
|
||||
if (e.code() == ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT)
|
||||
@ -163,24 +163,22 @@ void TCPHandler::runImpl()
|
||||
* Хотя в одном из них, мы должны отправить эксепшен клиенту, а в другом - не можем.
|
||||
* Будем пытаться отправить эксепшен клиенту в любом случае - см. ниже.
|
||||
*/
|
||||
LOG_ERROR(log, "Poco::Net::NetException. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
|
||||
<< ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what());
|
||||
state.io.onException();
|
||||
exception = new Exception(e.displayText(), ErrorCodes::POCO_EXCEPTION);
|
||||
}
|
||||
catch (const Poco::Exception & e)
|
||||
{
|
||||
LOG_ERROR(log, "Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
|
||||
<< ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what());
|
||||
state.io.onException();
|
||||
exception = new Exception(e.displayText(), ErrorCodes::POCO_EXCEPTION);
|
||||
}
|
||||
catch (const std::exception & e)
|
||||
{
|
||||
LOG_ERROR(log, "std::exception. Code: " << ErrorCodes::STD_EXCEPTION << ", e.what() = " << e.what());
|
||||
state.io.onException();
|
||||
exception = new Exception(e.what(), ErrorCodes::STD_EXCEPTION);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LOG_ERROR(log, "Unknown exception. Code: " << ErrorCodes::UNKNOWN_EXCEPTION);
|
||||
state.io.onException();
|
||||
exception = new Exception("Unknown exception", ErrorCodes::UNKNOWN_EXCEPTION);
|
||||
}
|
||||
|
||||
@ -285,11 +283,6 @@ void TCPHandler::processOrdinaryQuery()
|
||||
AsynchronousBlockInputStream async_in(state.io.in);
|
||||
async_in.readPrefix();
|
||||
|
||||
std::stringstream query_pipeline;
|
||||
async_in.dumpTree(query_pipeline);
|
||||
LOG_DEBUG(log, "Query pipeline:\n" << query_pipeline.rdbuf());
|
||||
|
||||
Stopwatch watch;
|
||||
while (true)
|
||||
{
|
||||
Block block;
|
||||
@ -341,9 +334,7 @@ void TCPHandler::processOrdinaryQuery()
|
||||
}
|
||||
|
||||
async_in.readSuffix();
|
||||
|
||||
watch.stop();
|
||||
logProfileInfo(watch, *state.io.in);
|
||||
state.io.onFinish();
|
||||
}
|
||||
}
|
||||
|
||||
@ -412,27 +403,6 @@ void TCPHandler::sendExtremes()
|
||||
}
|
||||
|
||||
|
||||
void TCPHandler::logProfileInfo(Stopwatch & watch, IBlockInputStream & in)
|
||||
{
|
||||
/// Выведем информацию о том, сколько считано строк и байт.
|
||||
size_t rows = 0;
|
||||
size_t bytes = 0;
|
||||
|
||||
in.getLeafRowsBytes(rows, bytes);
|
||||
|
||||
if (rows != 0)
|
||||
{
|
||||
LOG_INFO(log, std::fixed << std::setprecision(3)
|
||||
<< "Read " << rows << " rows, " << bytes / 1048576.0 << " MiB in " << watch.elapsedSeconds() << " sec., "
|
||||
<< static_cast<size_t>(rows / watch.elapsedSeconds()) << " rows/sec., " << bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.");
|
||||
}
|
||||
|
||||
QuotaForIntervals & quota = query_context.getQuota();
|
||||
if (!quota.empty())
|
||||
LOG_INFO(log, "Quota:\n" << quota.toString());
|
||||
}
|
||||
|
||||
|
||||
void TCPHandler::receiveHello()
|
||||
{
|
||||
/// Получить hello пакет.
|
||||
@ -568,10 +538,6 @@ void TCPHandler::receiveQuery()
|
||||
state.compression = Protocol::Compression::Enum(compression);
|
||||
|
||||
readStringBinary(state.query, *in);
|
||||
|
||||
LOG_DEBUG(log, "Query ID: " << state.query_id);
|
||||
LOG_DEBUG(log, "Query: " << state.query);
|
||||
LOG_DEBUG(log, "Requested stage: " << QueryProcessingStage::toString(stage));
|
||||
}
|
||||
|
||||
|
||||
|
@ -130,9 +130,6 @@ private:
|
||||
|
||||
/// Эта функция вызывается из разных потоков.
|
||||
void updateProgress(const Progress & value);
|
||||
|
||||
/// Вывести информацию о скорости выполнения SELECT запроса.
|
||||
void logProfileInfo(Stopwatch & watch, IBlockInputStream & in);
|
||||
};
|
||||
|
||||
|
||||
|
@ -340,8 +340,10 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads(
|
||||
const Names & virt_columns,
|
||||
const Settings & settings)
|
||||
{
|
||||
size_t min_marks_for_concurrent_read = (settings.merge_tree_min_rows_for_concurrent_read + data.index_granularity - 1) / data.index_granularity;
|
||||
size_t max_marks_to_use_cache = (settings.merge_tree_max_rows_to_use_cache + data.index_granularity - 1) / data.index_granularity;
|
||||
const size_t min_marks_for_concurrent_read =
|
||||
(settings.merge_tree_min_rows_for_concurrent_read + data.index_granularity - 1) / data.index_granularity;
|
||||
const size_t max_marks_to_use_cache =
|
||||
(settings.merge_tree_max_rows_to_use_cache + data.index_granularity - 1) / data.index_granularity;
|
||||
|
||||
/// На всякий случай перемешаем куски.
|
||||
std::random_shuffle(parts.begin(), parts.end());
|
||||
@ -354,12 +356,9 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads(
|
||||
/// Пусть отрезки будут перечислены справа налево, чтобы можно было выбрасывать самый левый отрезок с помощью pop_back().
|
||||
std::reverse(parts[i].ranges.begin(), parts[i].ranges.end());
|
||||
|
||||
sum_marks_in_parts[i] = 0;
|
||||
for (size_t j = 0; j < parts[i].ranges.size(); ++j)
|
||||
{
|
||||
MarkRange & range = parts[i].ranges[j];
|
||||
for (const auto & range : parts[i].ranges)
|
||||
sum_marks_in_parts[i] += range.end - range.begin;
|
||||
}
|
||||
|
||||
sum_marks += sum_marks_in_parts[i];
|
||||
}
|
||||
|
||||
@ -370,7 +369,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads(
|
||||
|
||||
if (sum_marks > 0)
|
||||
{
|
||||
size_t min_marks_per_thread = (sum_marks - 1) / threads + 1;
|
||||
const size_t min_marks_per_thread = (sum_marks - 1) / threads + 1;
|
||||
|
||||
for (size_t i = 0; i < threads && !parts.empty(); ++i)
|
||||
{
|
||||
@ -415,10 +414,11 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads(
|
||||
throw Exception("Unexpected end of ranges while spreading marks among threads", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
MarkRange & range = part.ranges.back();
|
||||
size_t marks_in_range = range.end - range.begin;
|
||||
|
||||
size_t marks_to_get_from_range = std::min(marks_in_range, need_marks);
|
||||
ranges_to_get_from_part.push_back(MarkRange(range.begin, range.begin + marks_to_get_from_range));
|
||||
const size_t marks_in_range = range.end - range.begin;
|
||||
const size_t marks_to_get_from_range = std::min(marks_in_range, need_marks);
|
||||
|
||||
ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range);
|
||||
range.begin += marks_to_get_from_range;
|
||||
marks_in_part -= marks_to_get_from_range;
|
||||
need_marks -= marks_to_get_from_range;
|
||||
|
@ -8,6 +8,136 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
/// Преобразование строки с датой или датой-с-временем в UInt64, содержащим числовое значение даты или даты-с-временем.
|
||||
UInt64 stringToDateOrDateTime(const String & s)
|
||||
{
|
||||
ReadBufferFromString in(s);
|
||||
|
||||
if (s.size() == strlen("YYYY-MM-DD"))
|
||||
{
|
||||
DayNum_t date{};
|
||||
readDateText(date, in);
|
||||
return UInt64(date);
|
||||
}
|
||||
else
|
||||
{
|
||||
time_t date_time{};
|
||||
readDateTimeText(date_time, in);
|
||||
if (!in.eof())
|
||||
throw Exception("String is too long for DateTime: " + s);
|
||||
return UInt64(date_time);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wsign-compare"
|
||||
|
||||
/** Более точное сравнение.
|
||||
* Отличается от Field::operator< и Field::operator== тем, что сравнивает значения разных числовых типов между собой.
|
||||
* Правила сравнения - такие же, что и в FunctionsComparison.
|
||||
* В том числе, сравнение знаковых и беззнаковых оставляем UB.
|
||||
*/
|
||||
class FieldVisitorAccurateEquals : public StaticVisitor<bool>
|
||||
{
|
||||
public:
|
||||
bool operator() (const Null & l, const Null & r) const { return true; }
|
||||
bool operator() (const Null & l, const UInt64 & r) const { return false; }
|
||||
bool operator() (const Null & l, const Int64 & r) const { return false; }
|
||||
bool operator() (const Null & l, const Float64 & r) const { return false; }
|
||||
bool operator() (const Null & l, const String & r) const { return false; }
|
||||
bool operator() (const Null & l, const Array & r) const { return false; }
|
||||
|
||||
bool operator() (const UInt64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const UInt64 & l, const UInt64 & r) const { return l == r; }
|
||||
bool operator() (const UInt64 & l, const Int64 & r) const { return l == r; }
|
||||
bool operator() (const UInt64 & l, const Float64 & r) const { return l == r; }
|
||||
bool operator() (const UInt64 & l, const String & r) const { return l == stringToDateOrDateTime(r); }
|
||||
bool operator() (const UInt64 & l, const Array & r) const { return false; }
|
||||
|
||||
bool operator() (const Int64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const Int64 & l, const UInt64 & r) const { return l == r; }
|
||||
bool operator() (const Int64 & l, const Int64 & r) const { return l == r; }
|
||||
bool operator() (const Int64 & l, const Float64 & r) const { return l == r; }
|
||||
bool operator() (const Int64 & l, const String & r) const { return false; }
|
||||
bool operator() (const Int64 & l, const Array & r) const { return false; }
|
||||
|
||||
bool operator() (const Float64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const Float64 & l, const UInt64 & r) const { return l == r; }
|
||||
bool operator() (const Float64 & l, const Int64 & r) const { return l == r; }
|
||||
bool operator() (const Float64 & l, const Float64 & r) const { return l == r; }
|
||||
bool operator() (const Float64 & l, const String & r) const { return false; }
|
||||
bool operator() (const Float64 & l, const Array & r) const { return false; }
|
||||
|
||||
bool operator() (const String & l, const Null & r) const { return false; }
|
||||
bool operator() (const String & l, const UInt64 & r) const { return stringToDateOrDateTime(l) == r; }
|
||||
bool operator() (const String & l, const Int64 & r) const { return false; }
|
||||
bool operator() (const String & l, const Float64 & r) const { return false; }
|
||||
bool operator() (const String & l, const String & r) const { return l == r; }
|
||||
bool operator() (const String & l, const Array & r) const { return false; }
|
||||
|
||||
bool operator() (const Array & l, const Null & r) const { return false; }
|
||||
bool operator() (const Array & l, const UInt64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const Int64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const Float64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const String & r) const { return false; }
|
||||
bool operator() (const Array & l, const Array & r) const { return l == r; }
|
||||
};
|
||||
|
||||
class FieldVisitorAccurateLess : public StaticVisitor<bool>
|
||||
{
|
||||
public:
|
||||
bool operator() (const Null & l, const Null & r) const { return false; }
|
||||
bool operator() (const Null & l, const UInt64 & r) const { return true; }
|
||||
bool operator() (const Null & l, const Int64 & r) const { return true; }
|
||||
bool operator() (const Null & l, const Float64 & r) const { return true; }
|
||||
bool operator() (const Null & l, const String & r) const { return true; }
|
||||
bool operator() (const Null & l, const Array & r) const { return true; }
|
||||
|
||||
bool operator() (const UInt64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const UInt64 & l, const UInt64 & r) const { return l < r; }
|
||||
bool operator() (const UInt64 & l, const Int64 & r) const { return l < r; }
|
||||
bool operator() (const UInt64 & l, const Float64 & r) const { return l < r; }
|
||||
bool operator() (const UInt64 & l, const String & r) const { return l < stringToDateOrDateTime(r); }
|
||||
bool operator() (const UInt64 & l, const Array & r) const { return true; }
|
||||
|
||||
bool operator() (const Int64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const Int64 & l, const UInt64 & r) const { return l < r; }
|
||||
bool operator() (const Int64 & l, const Int64 & r) const { return l < r; }
|
||||
bool operator() (const Int64 & l, const Float64 & r) const { return l < r; }
|
||||
bool operator() (const Int64 & l, const String & r) const { return true; }
|
||||
bool operator() (const Int64 & l, const Array & r) const { return true; }
|
||||
|
||||
bool operator() (const Float64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const Float64 & l, const UInt64 & r) const { return l < r; }
|
||||
bool operator() (const Float64 & l, const Int64 & r) const { return l < r; }
|
||||
bool operator() (const Float64 & l, const Float64 & r) const { return l < r; }
|
||||
bool operator() (const Float64 & l, const String & r) const { return true; }
|
||||
bool operator() (const Float64 & l, const Array & r) const { return true; }
|
||||
|
||||
bool operator() (const String & l, const Null & r) const { return false; }
|
||||
bool operator() (const String & l, const UInt64 & r) const { return stringToDateOrDateTime(l) < r; }
|
||||
bool operator() (const String & l, const Int64 & r) const { return false; }
|
||||
bool operator() (const String & l, const Float64 & r) const { return false; }
|
||||
bool operator() (const String & l, const String & r) const { return l < r; }
|
||||
bool operator() (const String & l, const Array & r) const { return true; }
|
||||
|
||||
bool operator() (const Array & l, const Null & r) const { return false; }
|
||||
bool operator() (const Array & l, const UInt64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const Int64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const Float64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const String & r) const { return false; }
|
||||
bool operator() (const Array & l, const Array & r) const { return l < r; }
|
||||
};
|
||||
|
||||
#pragma GCC diagnostic pop
|
||||
|
||||
|
||||
inline bool Range::equals(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateEquals(), lhs, rhs); }
|
||||
inline bool Range::less(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateLess(), lhs, rhs); }
|
||||
|
||||
|
||||
PKCondition::PKCondition(ASTPtr query, const Context & context_, const NamesAndTypesList & all_columns, const SortDescription & sort_descr_)
|
||||
: sort_descr(sort_descr_)
|
||||
{
|
||||
|
@ -130,7 +130,7 @@ BlockInputStreams StorageBuffer::read(
|
||||
*/
|
||||
if (processed_stage > QueryProcessingStage::FetchColumns)
|
||||
for (auto & stream : streams_from_buffers)
|
||||
stream = InterpreterSelectQuery(query, context, processed_stage, 0, stream).execute();
|
||||
stream = InterpreterSelectQuery(query, context, processed_stage, 0, stream).execute().in;
|
||||
|
||||
streams_from_dst.insert(streams_from_dst.end(), streams_from_buffers.begin(), streams_from_buffers.end());
|
||||
return streams_from_dst;
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user