mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-20 06:32:08 +00:00
Merge
This commit is contained in:
commit
d5e7f57b61
@ -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 apt-get install sysstat pstack mcelog
|
||||||
sudo dpkg -i vertica_7.1.1-0_amd64.deb
|
sudo dpkg -i vertica_7.1.1-0_amd64.deb
|
||||||
sudo sh -c "echo 'export TZ=Europe/Moscow' >> /home/dbadmin/.bash_profile"
|
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 /opt/vertica/sbin/install_vertica --hosts=127.0.0.1 --failure-threshold=NONE
|
||||||
sudo mkdir /opt/vertica-data/
|
sudo mkdir /opt/vertica-data/
|
||||||
sudo chown dbadmin /opt/vertica-data/
|
sudo chown dbadmin /opt/vertica-data/
|
||||||
@ -23,8 +24,6 @@ both directories: /opt/vertica-data/
|
|||||||
main menu
|
main menu
|
||||||
exit
|
exit
|
||||||
|
|
||||||
PS. Note that Vertica doesn't support IPv6.
|
|
||||||
|
|
||||||
How to prepare data
|
How to prepare data
|
||||||
-------------------
|
-------------------
|
||||||
|
|
||||||
|
@ -26,13 +26,13 @@ class IConnectionPool : private boost::noncopyable
|
|||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
typedef PoolBase<Connection>::Entry Entry;
|
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) };
|
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)
|
if (settings)
|
||||||
return Base::get(settings->queue_max_wait_ms.totalMilliseconds());
|
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:
|
public:
|
||||||
typedef IConnectionPool::Entry Entry;
|
typedef IConnectionPool::Entry Entry;
|
||||||
typedef PoolWithFailoverBase<IConnectionPool, Settings *> Base;
|
typedef PoolWithFailoverBase<IConnectionPool> Base;
|
||||||
|
|
||||||
ConnectionPoolWithFailover(ConnectionPools & nested_pools_,
|
ConnectionPoolWithFailover(ConnectionPools & nested_pools_,
|
||||||
LoadBalancing load_balancing,
|
LoadBalancing load_balancing,
|
||||||
@ -52,7 +52,7 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
/** Выделяет соединение для работы. */
|
/** Выделяет соединение для работы. */
|
||||||
Entry get(Settings * settings = nullptr) override
|
Entry get(const Settings * settings = nullptr) override
|
||||||
{
|
{
|
||||||
applyLoadBalancing(settings);
|
applyLoadBalancing(settings);
|
||||||
return Base::get(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);
|
applyLoadBalancing(settings);
|
||||||
return Base::getMany(settings);
|
return Base::getMany(settings);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected:
|
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
|
try
|
||||||
{
|
{
|
||||||
@ -90,7 +90,7 @@ private:
|
|||||||
std::vector<size_t> hostname_differences; /// Расстояния от имени этого хоста до имен хостов пулов.
|
std::vector<size_t> hostname_differences; /// Расстояния от имени этого хоста до имен хостов пулов.
|
||||||
LoadBalancing default_load_balancing;
|
LoadBalancing default_load_balancing;
|
||||||
|
|
||||||
void applyLoadBalancing(Settings * settings)
|
void applyLoadBalancing(const Settings * settings)
|
||||||
{
|
{
|
||||||
LoadBalancing load_balancing = default_load_balancing;
|
LoadBalancing load_balancing = default_load_balancing;
|
||||||
if (settings)
|
if (settings)
|
||||||
|
@ -21,17 +21,17 @@ class ParallelReplicas final : private boost::noncopyable
|
|||||||
{
|
{
|
||||||
public:
|
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);
|
void sendExternalTablesData(std::vector<ExternalTablesData> & data);
|
||||||
|
|
||||||
/// Отправить запрос на реплики.
|
/// Отправить запрос на реплики.
|
||||||
void sendQuery(const String & query, const String & query_id = "",
|
void sendQuery(const String & query, const String & query_id = "",
|
||||||
UInt64 stage = QueryProcessingStage::Complete, bool with_pending_data = false);
|
UInt64 stage = QueryProcessingStage::Complete, bool with_pending_data = false);
|
||||||
|
|
||||||
/// Получить пакет от какой-нибудь реплики.
|
/// Получить пакет от какой-нибудь реплики.
|
||||||
Connection::Packet receivePacket();
|
Connection::Packet receivePacket();
|
||||||
@ -82,7 +82,7 @@ private:
|
|||||||
void invalidateReplica(ReplicaMap::iterator it);
|
void invalidateReplica(ReplicaMap::iterator it);
|
||||||
|
|
||||||
private:
|
private:
|
||||||
Settings * settings;
|
const Settings * settings;
|
||||||
ReplicaMap replica_map;
|
ReplicaMap replica_map;
|
||||||
|
|
||||||
/// Если не nullptr, то используется, чтобы ограничить сетевой трафик.
|
/// Если не nullptr, то используется, чтобы ограничить сетевой трафик.
|
||||||
|
@ -24,36 +24,51 @@ public:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
/** шаблон для столбцов-констант (столбцов одинаковых значений).
|
/** Столбец-константа может содержать внутри себя само значение,
|
||||||
|
* или, в случае массивов, SharedPtr от значения-массива,
|
||||||
|
* чтобы избежать проблем производительности при копировании очень больших массивов.
|
||||||
|
*
|
||||||
|
* T - тип значения,
|
||||||
|
* DataHolder - как значение хранится в таблице (либо T, либо SharedPtr<T>)
|
||||||
|
* Derived должен реализовать методы getDataFromHolderImpl - получить ссылку на значение из holder-а.
|
||||||
|
*
|
||||||
|
* Для строк и массивов реализации sizeOfField и byteSize могут быть некорректными.
|
||||||
*/
|
*/
|
||||||
template <typename T>
|
template <typename T, typename DataHolder, typename Derived>
|
||||||
class ColumnConst final : public IColumnConst
|
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:
|
public:
|
||||||
typedef T Type;
|
typedef T Type;
|
||||||
typedef typename NearestFieldType<T>::Type FieldType;
|
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() + ">"; }
|
std::string getName() const override { return "ColumnConst<" + TypeName<T>::get() + ">"; }
|
||||||
bool isNumeric() const override { return IsNumber<T>::value; }
|
bool isNumeric() const override { return IsNumber<T>::value; }
|
||||||
bool isFixed() const override { return IsNumber<T>::value; }
|
bool isFixed() const override { return IsNumber<T>::value; }
|
||||||
size_t sizeOfField() const override { return sizeof(T); }
|
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; }
|
size_t size() const override { return s; }
|
||||||
Field operator[](size_t n) const override { return FieldType(data); }
|
Field operator[](size_t n) const override { return FieldType(getDataFromHolder()); }
|
||||||
void get(size_t n, Field & res) const override { res = FieldType(data); }
|
void get(size_t n, Field & res) const override { res = FieldType(getDataFromHolder()); }
|
||||||
|
|
||||||
ColumnPtr cut(size_t start, size_t length) const override
|
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
|
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(),
|
throw Exception("Cannot insert different element into constant column " + getName(),
|
||||||
ErrorCodes::CANNOT_INSERT_ELEMENT_INTO_CONSTANT_COLUMN);
|
ErrorCodes::CANNOT_INSERT_ELEMENT_INTO_CONSTANT_COLUMN);
|
||||||
++s;
|
++s;
|
||||||
@ -66,7 +81,7 @@ public:
|
|||||||
|
|
||||||
void insertFrom(const IColumn & src, size_t n) override
|
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(),
|
throw Exception("Cannot insert different element into constant column " + getName(),
|
||||||
ErrorCodes::CANNOT_INSERT_ELEMENT_INTO_CONSTANT_COLUMN);
|
ErrorCodes::CANNOT_INSERT_ELEMENT_INTO_CONSTANT_COLUMN);
|
||||||
++s;
|
++s;
|
||||||
@ -79,7 +94,7 @@ public:
|
|||||||
if (s != filt.size())
|
if (s != filt.size())
|
||||||
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
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
|
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);
|
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();
|
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); }
|
size_t byteSize() const override { return sizeof(data) + sizeof(s); }
|
||||||
@ -103,13 +118,13 @@ public:
|
|||||||
if (perm.size() < limit)
|
if (perm.size() < limit)
|
||||||
throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
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
|
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_);
|
const Derived & rhs = static_cast<const Derived &>(rhs_);
|
||||||
return data < rhs.data /// TODO: правильное сравнение NaN-ов в константных столбцах.
|
return getDataFromHolder() < rhs.getDataFromHolder() /// TODO: правильное сравнение NaN-ов в константных столбцах.
|
||||||
? -1
|
? -1
|
||||||
: (data == rhs.data
|
: (data == rhs.data
|
||||||
? 0
|
? 0
|
||||||
@ -123,30 +138,79 @@ public:
|
|||||||
res[i] = i;
|
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 getDataAt(size_t n) const override;
|
||||||
StringRef getDataAtWithTerminatingZero(size_t n) const override;
|
StringRef getDataAtWithTerminatingZero(size_t n) const override;
|
||||||
UInt64 get64(size_t n) const override;
|
UInt64 get64(size_t n) const override;
|
||||||
|
|
||||||
/** Более эффективные методы манипуляции */
|
/** Более эффективные методы манипуляции */
|
||||||
T & getData() { return data; }
|
T & getData() { return this->data; }
|
||||||
const T & getData() const { return data; }
|
const T & getData() const { return this->data; }
|
||||||
|
|
||||||
/** Преобразование из константы в полноценный столбец */
|
/** Преобразование из константы в полноценный столбец */
|
||||||
ColumnPtr convertToFullColumn() const override;
|
ColumnPtr convertToFullColumn() const override;
|
||||||
|
|
||||||
void getExtremes(Field & min, Field & max) const override
|
void getExtremes(Field & min, Field & max) const override
|
||||||
{
|
{
|
||||||
min = FieldType(data);
|
min = typename ColumnConstBase<T, T, ColumnConst<T>>::FieldType(this->data);
|
||||||
max = FieldType(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:
|
private:
|
||||||
size_t s;
|
friend class ColumnConstBase<Array, SharedPtr<Array>, ColumnConst<Array>>;
|
||||||
T data;
|
|
||||||
DataTypePtr data_type;
|
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>;
|
ColumnVector<T> * res_ = new ColumnVector<T>;
|
||||||
ColumnPtr res = res_;
|
ColumnPtr res = res_;
|
||||||
res_->getData().assign(s, data);
|
res_->getData().assign(this->s, this->data);
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
template <> ColumnPtr ColumnConst<String>::convertToFullColumn() const;
|
template <> ColumnPtr ColumnConst<String>::convertToFullColumn() const;
|
||||||
|
|
||||||
template <> ColumnPtr ColumnConst<Array>::convertToFullColumn() const;
|
|
||||||
|
|
||||||
|
|
||||||
template <typename T> StringRef ColumnConst<T>::getDataAt(size_t n) 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
|
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
|
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);
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Для элементарных типов.
|
/// Для элементарных типов.
|
||||||
|
@ -38,6 +38,11 @@ public:
|
|||||||
{
|
{
|
||||||
return amount;
|
return amount;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Int64 getPeak() const
|
||||||
|
{
|
||||||
|
return peak;
|
||||||
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
@ -13,6 +13,7 @@
|
|||||||
|
|
||||||
#include <stdint.h>
|
#include <stdint.h>
|
||||||
#include <cstdlib>
|
#include <cstdlib>
|
||||||
|
#include <stddef.h>
|
||||||
|
|
||||||
#define ROTL(x,b) (u64)( ((x) << (b)) | ( (x) >> (64 - (b))) )
|
#define ROTL(x,b) (u64)( ((x) << (b)) | ( (x) >> (64 - (b))) )
|
||||||
|
|
||||||
|
@ -78,5 +78,7 @@
|
|||||||
/// Граница, на которых должны быть выровнены блоки для асинхронных файловых операций.
|
/// Граница, на которых должны быть выровнены блоки для асинхронных файловых операций.
|
||||||
#define DEFAULT_AIO_FILE_BLOCK_SIZE 4096
|
#define DEFAULT_AIO_FILE_BLOCK_SIZE 4096
|
||||||
|
|
||||||
|
#define DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS_STR "7500"
|
||||||
|
|
||||||
#define ALWAYS_INLINE __attribute__((__always_inline__))
|
#define ALWAYS_INLINE __attribute__((__always_inline__))
|
||||||
#define NO_INLINE __attribute__((__noinline__))
|
#define NO_INLINE __attribute__((__noinline__))
|
||||||
|
@ -31,6 +31,8 @@ ExceptionPtr cloneCurrentException();
|
|||||||
void tryLogCurrentException(const char * log_name);
|
void tryLogCurrentException(const char * log_name);
|
||||||
void tryLogCurrentException(Poco::Logger * logger);
|
void tryLogCurrentException(Poco::Logger * logger);
|
||||||
|
|
||||||
|
std::string getCurrentExceptionMessage(bool with_stacktrace);
|
||||||
|
|
||||||
|
|
||||||
void rethrowFirstException(Exceptions & exceptions);
|
void rethrowFirstException(Exceptions & exceptions);
|
||||||
|
|
||||||
|
@ -24,6 +24,25 @@ struct BlockIO
|
|||||||
Block in_sample; /// Пример блока, который будет прочитан из in.
|
Block in_sample; /// Пример блока, который будет прочитан из in.
|
||||||
Block out_sample; /// Пример блока, которого нужно писать в out.
|
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)
|
BlockIO & operator= (const BlockIO & rhs)
|
||||||
{
|
{
|
||||||
/// Обеспечиваем правильный порядок уничтожения.
|
/// Обеспечиваем правильный порядок уничтожения.
|
||||||
@ -37,6 +56,9 @@ struct BlockIO
|
|||||||
in_sample = rhs.in_sample;
|
in_sample = rhs.in_sample;
|
||||||
out_sample = rhs.out_sample;
|
out_sample = rhs.out_sample;
|
||||||
|
|
||||||
|
finish_callback = rhs.finish_callback;
|
||||||
|
exception_callback = rhs.exception_callback;
|
||||||
|
|
||||||
return *this;
|
return *this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -136,6 +136,9 @@ protected:
|
|||||||
{
|
{
|
||||||
createParallelReplicas();
|
createParallelReplicas();
|
||||||
|
|
||||||
|
if (settings.skip_unavailable_shards && 0 == parallel_replicas->size())
|
||||||
|
return Block();
|
||||||
|
|
||||||
established = true;
|
established = true;
|
||||||
|
|
||||||
parallel_replicas->sendQuery(query, "", stage, true);
|
parallel_replicas->sendQuery(query, "", stage, true);
|
||||||
|
@ -15,7 +15,7 @@ namespace DB
|
|||||||
class RemoteBlockOutputStream : public IBlockOutputStream
|
class RemoteBlockOutputStream : public IBlockOutputStream
|
||||||
{
|
{
|
||||||
public:
|
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_)
|
: connection(connection_), query(query_), settings(settings_)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
@ -105,7 +105,7 @@ public:
|
|||||||
private:
|
private:
|
||||||
Connection & connection;
|
Connection & connection;
|
||||||
String query;
|
String query;
|
||||||
Settings * settings;
|
const Settings * settings;
|
||||||
Block sample_block;
|
Block sample_block;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -4,20 +4,25 @@
|
|||||||
#include <DB/DataTypes/DataTypeString.h>
|
#include <DB/DataTypes/DataTypeString.h>
|
||||||
#include <DB/DataTypes/DataTypeArray.h>
|
#include <DB/DataTypes/DataTypeArray.h>
|
||||||
|
|
||||||
|
#include <DB/Core/Exception.h>
|
||||||
|
#include <DB/Core/ErrorCodes.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
/// Для заданного значения Field возвращает минимальный тип данных, позволяющий хранить значение этого типа.
|
/** Для заданного значения Field возвращает минимальный тип данных, позволяющий хранить значение этого типа.
|
||||||
|
* В случае, если Field - массив, конвертирует все элементы к общему типу.
|
||||||
|
*/
|
||||||
class FieldToDataType : public StaticVisitor<DataTypePtr>
|
class FieldToDataType : public StaticVisitor<DataTypePtr>
|
||||||
{
|
{
|
||||||
public:
|
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<UInt8>::max()) return new DataTypeUInt8;
|
||||||
if (x <= std::numeric_limits<UInt16>::max()) return new DataTypeUInt16;
|
if (x <= std::numeric_limits<UInt16>::max()) return new DataTypeUInt16;
|
||||||
@ -25,7 +30,7 @@ public:
|
|||||||
return new DataTypeUInt64;
|
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<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;
|
if (x <= std::numeric_limits<Int16>::max() && x >= std::numeric_limits<Int16>::min()) return new DataTypeInt16;
|
||||||
@ -33,22 +38,18 @@ public:
|
|||||||
return new DataTypeInt64;
|
return new DataTypeInt64;
|
||||||
}
|
}
|
||||||
|
|
||||||
DataTypePtr operator() (const Float64 & x) const
|
DataTypePtr operator() (Float64 & x) const
|
||||||
{
|
{
|
||||||
return new DataTypeFloat64;
|
return new DataTypeFloat64;
|
||||||
}
|
}
|
||||||
|
|
||||||
DataTypePtr operator() (const String & x) const
|
DataTypePtr operator() (String & x) const
|
||||||
{
|
{
|
||||||
return new DataTypeString;
|
return new DataTypeString;
|
||||||
}
|
}
|
||||||
|
|
||||||
DataTypePtr operator() (const Array & x) const
|
DataTypePtr operator() (Array & x) const;
|
||||||
{
|
|
||||||
return new DataTypeArray(apply_visitor(FieldToDataType(), x.at(0)));
|
|
||||||
}
|
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -41,7 +41,9 @@ struct ArrayMapImpl
|
|||||||
|
|
||||||
static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped)
|
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());
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -59,9 +61,20 @@ struct ArrayFilterImpl
|
|||||||
/// Если массивов несколько, сюда передается первый.
|
/// Если массивов несколько, сюда передается первый.
|
||||||
static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped)
|
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)
|
if (!column_filter)
|
||||||
throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN);
|
{
|
||||||
|
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();
|
const IColumn::Filter & filter = column_filter->getData();
|
||||||
ColumnPtr filtered = array->getData().filter(filter);
|
ColumnPtr filtered = array->getData().filter(filter);
|
||||||
@ -100,9 +113,34 @@ struct ArrayCountImpl
|
|||||||
|
|
||||||
static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped)
|
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)
|
if (!column_filter)
|
||||||
throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN);
|
{
|
||||||
|
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::Filter & filter = column_filter->getData();
|
||||||
const IColumn::Offsets_t & offsets = array->getOffsets();
|
const IColumn::Offsets_t & offsets = array->getOffsets();
|
||||||
@ -139,9 +177,34 @@ struct ArrayExistsImpl
|
|||||||
|
|
||||||
static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped)
|
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)
|
if (!column_filter)
|
||||||
throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN);
|
{
|
||||||
|
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::Filter & filter = column_filter->getData();
|
||||||
const IColumn::Offsets_t & offsets = array->getOffsets();
|
const IColumn::Offsets_t & offsets = array->getOffsets();
|
||||||
@ -182,9 +245,34 @@ struct ArrayAllImpl
|
|||||||
|
|
||||||
static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped)
|
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)
|
if (!column_filter)
|
||||||
throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN);
|
{
|
||||||
|
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::Filter & filter = column_filter->getData();
|
||||||
const IColumn::Offsets_t & offsets = array->getOffsets();
|
const IColumn::Offsets_t & offsets = array->getOffsets();
|
||||||
@ -245,7 +333,27 @@ struct ArraySumImpl
|
|||||||
const ColumnVector<Element> * column = typeid_cast<const ColumnVector<Element> *>(&*mapped);
|
const ColumnVector<Element> * column = typeid_cast<const ColumnVector<Element> *>(&*mapped);
|
||||||
|
|
||||||
if (!column)
|
if (!column)
|
||||||
return false;
|
{
|
||||||
|
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();
|
const typename ColumnVector<Element>::Container_t & data = column->getData();
|
||||||
ColumnVector<Result> * res_column = new ColumnVector<Result>(offsets.size());
|
ColumnVector<Result> * res_column = new ColumnVector<Result>(offsets.size());
|
||||||
@ -300,9 +408,41 @@ struct ArrayFirstImpl
|
|||||||
|
|
||||||
static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped)
|
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)
|
if (!column_filter)
|
||||||
throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN);
|
{
|
||||||
|
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 & filter = column_filter->getData();
|
||||||
const auto & offsets = array->getOffsets();
|
const auto & offsets = array->getOffsets();
|
||||||
@ -345,9 +485,34 @@ struct ArrayFirstIndexImpl
|
|||||||
|
|
||||||
static ColumnPtr execute(const ColumnArray * array, ColumnPtr mapped)
|
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)
|
if (!column_filter)
|
||||||
throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN);
|
{
|
||||||
|
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 & filter = column_filter->getData();
|
||||||
const auto & offsets = array->getOffsets();
|
const auto & offsets = array->getOffsets();
|
||||||
|
@ -89,11 +89,12 @@ namespace DB
|
|||||||
|
|
||||||
static inline ResultType apply(A x)
|
static inline ResultType apply(A x)
|
||||||
{
|
{
|
||||||
return x < 18 ? 0
|
return x < 1 ? 0
|
||||||
|
: (x < 18 ? 17
|
||||||
: (x < 25 ? 18
|
: (x < 25 ? 18
|
||||||
: (x < 35 ? 25
|
: (x < 35 ? 25
|
||||||
: (x < 45 ? 35
|
: (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]);
|
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;
|
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)
|
if (required_substring_is_prefix)
|
||||||
res[i] = revert ^ regexp->getRE2()->Match(
|
res[i] = revert ^ regexp->getRE2()->Match(
|
||||||
re2_st::StringPiece(str_data, str_size),
|
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
|
else
|
||||||
res[i] = revert ^ regexp->getRE2()->Match(
|
res[i] = revert ^ regexp->getRE2()->Match(
|
||||||
re2_st::StringPiece(str_data, str_size),
|
re2_st::StringPiece(str_data, str_size),
|
||||||
|
@ -84,7 +84,7 @@ public:
|
|||||||
|
|
||||||
void ignore(size_t n)
|
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);
|
size_t bytes_to_ignore = std::min(static_cast<size_t>(working_buffer.end() - pos), n);
|
||||||
pos += bytes_to_ignore;
|
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 assertString(const char * s, ReadBuffer & buf);
|
||||||
void assertEOF(ReadBuffer & buf);
|
void assertEOF(ReadBuffer & buf);
|
||||||
|
void assertChar(char symbol, ReadBuffer & buf);
|
||||||
|
|
||||||
inline void assertString(const String & s, 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 раза)
|
/// strpbrk в libc под Linux на процессорах с SSE 4.2 хорошо оптимизирована (этот if ускоряет код в 1.5 раза)
|
||||||
if (nullptr == strpbrk(s.data(), "\b\f\n\r\t\'\\") && strlen(s.data()) == s.size())
|
if (nullptr == strpbrk(str, "\b\f\n\r\t\'\\") && strlen(str) == size)
|
||||||
writeString(s, buf);
|
writeString(str, size, buf);
|
||||||
else
|
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 Float64 & x, WriteBuffer & buf) { writeFloatText(x, buf); }
|
||||||
inline void writeText(const String & x, WriteBuffer & buf) { writeEscapedString(x, buf); }
|
inline void writeText(const String & x, WriteBuffer & buf) { writeEscapedString(x, buf); }
|
||||||
inline void writeText(const bool & x, WriteBuffer & buf) { writeBoolText(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 VisitID_t & x, WriteBuffer & buf) { writeIntText(static_cast<const UInt64 &>(x), buf); }
|
||||||
inline void writeText(const mysqlxx::Date & x, WriteBuffer & buf) { writeDateText(x, buf); }
|
inline void writeText(const mysqlxx::Date & x, WriteBuffer & buf) { writeDateText(x, buf); }
|
||||||
|
@ -40,6 +40,7 @@ class ProcessListElement;
|
|||||||
class Macros;
|
class Macros;
|
||||||
class Progress;
|
class Progress;
|
||||||
class Clusters;
|
class Clusters;
|
||||||
|
class QueryLog;
|
||||||
|
|
||||||
|
|
||||||
/// имя таблицы -> таблица
|
/// имя таблицы -> таблица
|
||||||
@ -64,12 +65,30 @@ typedef std::vector<DatabaseAndTableName> Dependencies;
|
|||||||
*/
|
*/
|
||||||
class Context
|
class Context
|
||||||
{
|
{
|
||||||
|
public:
|
||||||
|
enum class Interface
|
||||||
|
{
|
||||||
|
TCP = 1,
|
||||||
|
HTTP = 2,
|
||||||
|
OLAP_HTTP = 3,
|
||||||
|
};
|
||||||
|
|
||||||
|
enum class HTTPMethod
|
||||||
|
{
|
||||||
|
UNKNOWN = 0,
|
||||||
|
GET = 1,
|
||||||
|
POST = 2,
|
||||||
|
};
|
||||||
|
|
||||||
private:
|
private:
|
||||||
typedef std::shared_ptr<ContextShared> Shared;
|
typedef std::shared_ptr<ContextShared> Shared;
|
||||||
Shared shared;
|
Shared shared;
|
||||||
|
|
||||||
String user; /// Текущий пользователь.
|
String user; /// Текущий пользователь.
|
||||||
Poco::Net::IPAddress ip_address; /// IP-адрес, с которого задан запрос.
|
Poco::Net::IPAddress ip_address; /// IP-адрес, с которого задан запрос.
|
||||||
|
Interface interface = Interface::TCP;
|
||||||
|
HTTPMethod http_method = HTTPMethod::UNKNOWN; /// NOTE Возможно, перенести это в отдельный struct ClientInfo.
|
||||||
|
|
||||||
std::shared_ptr<QuotaForIntervals> quota; /// Текущая квота. По-умолчанию - пустая квота, которая ничего не ограничивает.
|
std::shared_ptr<QuotaForIntervals> quota; /// Текущая квота. По-умолчанию - пустая квота, которая ничего не ограничивает.
|
||||||
String current_database; /// Текущая БД.
|
String current_database; /// Текущая БД.
|
||||||
String current_query_id; /// Id текущего запроса.
|
String current_query_id; /// Id текущего запроса.
|
||||||
@ -108,6 +127,12 @@ public:
|
|||||||
String getUser() const { return user; }
|
String getUser() const { return user; }
|
||||||
Poco::Net::IPAddress getIPAddress() const { return ip_address; }
|
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);
|
void setQuota(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & address);
|
||||||
QuotaForIntervals & getQuota();
|
QuotaForIntervals & getQuota();
|
||||||
|
|
||||||
@ -242,6 +267,8 @@ public:
|
|||||||
|
|
||||||
Compiler & getCompiler();
|
Compiler & getCompiler();
|
||||||
|
|
||||||
|
QueryLog & getQueryLog();
|
||||||
|
|
||||||
/// Позволяет выбрать метод сжатия по условиям, описанным в конфигурационном файле.
|
/// Позволяет выбрать метод сжатия по условиям, описанным в конфигурационном файле.
|
||||||
CompressionMethod chooseCompressionMethod(size_t part_size, double part_size_ratio) const;
|
CompressionMethod chooseCompressionMethod(size_t part_size, double part_size_ratio) const;
|
||||||
|
|
||||||
|
@ -5,8 +5,8 @@
|
|||||||
#include <Poco/Net/IPAddress.h>
|
#include <Poco/Net/IPAddress.h>
|
||||||
#include <DB/Core/Types.h>
|
#include <DB/Core/Types.h>
|
||||||
#include <DB/Common/ConcurrentBoundedQueue.h>
|
#include <DB/Common/ConcurrentBoundedQueue.h>
|
||||||
|
#include <DB/Storages/IStorage.h>
|
||||||
#include <DB/Interpreters/Context.h>
|
#include <DB/Interpreters/Context.h>
|
||||||
#include <statdaemons/Stopwatch.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -34,40 +34,33 @@ struct QueryLogElement
|
|||||||
SHUTDOWN = 0, /// Эта запись имеет служебное значение.
|
SHUTDOWN = 0, /// Эта запись имеет служебное значение.
|
||||||
QUERY_START = 1,
|
QUERY_START = 1,
|
||||||
QUERY_FINISH = 2,
|
QUERY_FINISH = 2,
|
||||||
|
EXCEPTION_BEFORE_START = 3,
|
||||||
|
EXCEPTION_WHILE_PROCESSING = 4,
|
||||||
};
|
};
|
||||||
|
|
||||||
enum Interface
|
Type type = QUERY_START;
|
||||||
{
|
|
||||||
TCP = 1,
|
|
||||||
HTTP = 2,
|
|
||||||
OLAP_HTTP = 3,
|
|
||||||
};
|
|
||||||
|
|
||||||
enum HTTPMethod
|
|
||||||
{
|
|
||||||
UNKNOWN = 0,
|
|
||||||
GET = 1,
|
|
||||||
POST = 2,
|
|
||||||
};
|
|
||||||
|
|
||||||
Type type;
|
|
||||||
|
|
||||||
/// В зависимости от типа, не все поля могут быть заполнены.
|
/// В зависимости от типа, не все поля могут быть заполнены.
|
||||||
|
|
||||||
time_t event_time;
|
time_t event_time{};
|
||||||
time_t query_start_time;
|
time_t query_start_time{};
|
||||||
UInt64 query_duration_ms;
|
UInt64 query_duration_ms{};
|
||||||
|
|
||||||
UInt64 read_rows;
|
UInt64 read_rows{};
|
||||||
UInt64 read_bytes;
|
UInt64 read_bytes{};
|
||||||
|
|
||||||
UInt64 result_rows;
|
UInt64 result_rows{};
|
||||||
UInt64 result_bytes;
|
UInt64 result_bytes{};
|
||||||
|
|
||||||
|
UInt64 memory_usage{};
|
||||||
|
|
||||||
String query;
|
String query;
|
||||||
|
|
||||||
Interface interface;
|
String exception;
|
||||||
HTTPMethod http_method;
|
String stack_trace;
|
||||||
|
|
||||||
|
Context::Interface interface = Context::Interface::TCP;
|
||||||
|
Context::HTTPMethod http_method = Context::HTTPMethod::UNKNOWN;
|
||||||
Poco::Net::IPAddress ip_address;
|
Poco::Net::IPAddress ip_address;
|
||||||
String user;
|
String user;
|
||||||
String query_id;
|
String query_id;
|
||||||
@ -76,6 +69,8 @@ struct QueryLogElement
|
|||||||
|
|
||||||
#define DBMS_QUERY_LOG_QUEUE_SIZE 1024
|
#define DBMS_QUERY_LOG_QUEUE_SIZE 1024
|
||||||
|
|
||||||
|
class Context;
|
||||||
|
|
||||||
|
|
||||||
class QueryLog : private boost::noncopyable
|
class QueryLog : private boost::noncopyable
|
||||||
{
|
{
|
||||||
@ -89,25 +84,8 @@ public:
|
|||||||
* где N - минимальное число, начиная с 1 такое, что таблицы с таким именем ещё нет;
|
* где N - минимальное число, начиная с 1 такое, что таблицы с таким именем ещё нет;
|
||||||
* и создаётся новая таблица, как будто существующей таблицы не было.
|
* и создаётся новая таблица, как будто существующей таблицы не было.
|
||||||
*/
|
*/
|
||||||
QueryLog(Context & context_, const String & database_name_, const String & table_name_, size_t flush_interval_milliseconds_)
|
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_)
|
~QueryLog();
|
||||||
{
|
|
||||||
data.reserve(DBMS_QUERY_LOG_QUEUE_SIZE);
|
|
||||||
|
|
||||||
// TODO
|
|
||||||
|
|
||||||
saving_thread = std::thread([this] { threadFunction(); });
|
|
||||||
}
|
|
||||||
|
|
||||||
~QueryLog()
|
|
||||||
{
|
|
||||||
/// Говорим потоку, что надо завершиться.
|
|
||||||
QueryLogElement elem;
|
|
||||||
elem.type = QueryLogElement::SHUTDOWN;
|
|
||||||
queue.push(elem);
|
|
||||||
|
|
||||||
saving_thread.join();
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Добавить запись в лог.
|
/** Добавить запись в лог.
|
||||||
* Сохранение в таблицу делается асинхронно, и в случае сбоя, запись может никуда не попасть.
|
* Сохранение в таблицу делается асинхронно, и в случае сбоя, запись может никуда не попасть.
|
||||||
@ -134,111 +112,15 @@ private:
|
|||||||
*/
|
*/
|
||||||
std::vector<QueryLogElement> data;
|
std::vector<QueryLogElement> data;
|
||||||
|
|
||||||
|
Logger * log {&Logger::get("QueryLog")};
|
||||||
|
|
||||||
/** В этом потоке данные вынимаются из queue, складываются в data, а затем вставляются в таблицу.
|
/** В этом потоке данные вынимаются из queue, складываются в data, а затем вставляются в таблицу.
|
||||||
*/
|
*/
|
||||||
std::thread saving_thread;
|
std::thread saving_thread;
|
||||||
|
|
||||||
|
void threadFunction();
|
||||||
void threadFunction()
|
static Block createBlock();
|
||||||
{
|
void flush();
|
||||||
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())
|
|
||||||
{
|
|
||||||
element = queue.pop();
|
|
||||||
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 createBlock()
|
|
||||||
{
|
|
||||||
return {
|
|
||||||
{new ColumnUInt8, new DataTypeUInt8, "type"},
|
|
||||||
{new ColumnUInt32, new DataTypeDateTime, "event_time"},
|
|
||||||
{new ColumnUInt32, new DataTypeDateTime, "query_start_time"},
|
|
||||||
};
|
|
||||||
|
|
||||||
/* 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;
|
|
||||||
|
|
||||||
String query;
|
|
||||||
|
|
||||||
Interface interface;
|
|
||||||
HTTPMethod http_method;
|
|
||||||
Poco::Net::IPAddress ip_address;
|
|
||||||
String user;
|
|
||||||
String query_id;*/
|
|
||||||
}
|
|
||||||
|
|
||||||
void flush()
|
|
||||||
{
|
|
||||||
try
|
|
||||||
{
|
|
||||||
Block block = createBlock();
|
|
||||||
|
|
||||||
// TODO Формирование блока и запись.
|
|
||||||
}
|
|
||||||
catch (...)
|
|
||||||
{
|
|
||||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
||||||
}
|
|
||||||
|
|
||||||
data.clear();
|
|
||||||
}
|
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
@ -83,7 +83,7 @@ struct QuotaForInterval
|
|||||||
|
|
||||||
/// Увеличить соответствующее значение.
|
/// Увеличить соответствующее значение.
|
||||||
void addQuery(time_t current_time, const String & quota_name);
|
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);
|
void checkExceeded(time_t current_time, const String & quota_name);
|
||||||
@ -139,7 +139,7 @@ public:
|
|||||||
void setMax(const QuotaForIntervals & quota);
|
void setMax(const QuotaForIntervals & quota);
|
||||||
|
|
||||||
void addQuery(time_t current_time);
|
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);
|
void checkExceeded(time_t current_time);
|
||||||
|
|
||||||
|
@ -97,6 +97,14 @@ struct Settings
|
|||||||
M(SettingUInt64, parallel_replicas_count, 0) \
|
M(SettingUInt64, parallel_replicas_count, 0) \
|
||||||
M(SettingUInt64, parallel_replica_offset, 0) \
|
M(SettingUInt64, parallel_replica_offset, 0) \
|
||||||
\
|
\
|
||||||
|
/** Тихо пропускать недоступные шарды. */ \
|
||||||
|
M(SettingBool, skip_unavailable_shards, false) \
|
||||||
|
\
|
||||||
|
/** Не мерджить состояния агрегации с разных серверов при распределённой обработке запроса \
|
||||||
|
* - на случай, когда доподлинно известно, что на разных шардах разные ключи. \
|
||||||
|
*/ \
|
||||||
|
M(SettingBool, distributed_group_by_no_merge, false) \
|
||||||
|
\
|
||||||
/** Тонкие настройки для чтения из MergeTree */ \
|
/** Тонкие настройки для чтения из MergeTree */ \
|
||||||
\
|
\
|
||||||
/** Если из одного файла читается хотя бы столько строк, чтение можно распараллелить. */ \
|
/** Если из одного файла читается хотя бы столько строк, чтение можно распараллелить. */ \
|
||||||
@ -135,6 +143,9 @@ struct Settings
|
|||||||
\
|
\
|
||||||
/** Приоритет запроса. 1 - самый высокий, больше - ниже; 0 - не использовать приоритеты. */ \
|
/** Приоритет запроса. 1 - самый высокий, больше - ниже; 0 - не использовать приоритеты. */ \
|
||||||
M(SettingUInt64, priority, 0) \
|
M(SettingUInt64, priority, 0) \
|
||||||
|
\
|
||||||
|
/** Логгировать запросы и писать лог в системную таблицу. */ \
|
||||||
|
M(SettingBool, log_queries, 0) \
|
||||||
|
|
||||||
/// Всевозможные ограничения на выполнение запроса.
|
/// Всевозможные ограничения на выполнение запроса.
|
||||||
Limits limits;
|
Limits limits;
|
||||||
|
@ -20,7 +20,7 @@ public:
|
|||||||
ASTLiteral() = default;
|
ASTLiteral() = default;
|
||||||
ASTLiteral(const StringRange range_, const Field & value_) : ASTWithAlias(range_), value(value_) {}
|
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); }
|
String getID() const override { return "Literal_" + apply_visitor(FieldVisitorDump(), value); }
|
||||||
|
@ -6,9 +6,8 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
|
||||||
/** Запрос с секцией FORMAT.
|
/** Запрос с секцией FORMAT.
|
||||||
*/
|
*/
|
||||||
class ASTQueryWithOutput : public IAST
|
class ASTQueryWithOutput : public IAST
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
@ -16,6 +15,11 @@ public:
|
|||||||
|
|
||||||
ASTQueryWithOutput() = default;
|
ASTQueryWithOutput() = default;
|
||||||
ASTQueryWithOutput(const StringRange range_) : IAST(range_) {}
|
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;
|
ASTPtr clone() const override;
|
||||||
|
|
||||||
|
/// Возвращает указатель на формат из последнего SELECT'а цепочки UNION ALL.
|
||||||
|
const IAST * getFormat() const override;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
bool distinct = false;
|
bool distinct = false;
|
||||||
ASTPtr select_expression_list;
|
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.
|
/** Литерал - одно из: NULL, UInt64, Int64, Float64, String.
|
||||||
*/
|
*/
|
||||||
class ParserLiteral : public IParserBase
|
class ParserLiteral : public IParserBase
|
||||||
|
@ -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
|
struct Range
|
||||||
{
|
{
|
||||||
private:
|
private:
|
||||||
static bool equals(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateEquals(), lhs, rhs); }
|
static bool equals(const Field & lhs, const Field & rhs);
|
||||||
static bool less(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateLess(), lhs, rhs); }
|
static bool less(const Field & lhs, const Field & rhs);
|
||||||
|
|
||||||
public:
|
public:
|
||||||
Field left; /// левая граница, если есть
|
Field left; /// левая граница, если есть
|
||||||
|
@ -96,6 +96,7 @@ private:
|
|||||||
};
|
};
|
||||||
|
|
||||||
bool is_interactive = true; /// Использовать readline интерфейс или batch режим.
|
bool is_interactive = true; /// Использовать readline интерфейс или batch режим.
|
||||||
|
bool need_render_progress = true; /// Рисовать прогресс выполнения запроса.
|
||||||
bool print_time_to_stderr = false; /// В неинтерактивном режиме, выводить время выполнения в stderr.
|
bool print_time_to_stderr = false; /// В неинтерактивном режиме, выводить время выполнения в stderr.
|
||||||
bool stdin_is_not_tty = false; /// stdin - не терминал.
|
bool stdin_is_not_tty = false; /// stdin - не терминал.
|
||||||
|
|
||||||
@ -268,6 +269,9 @@ private:
|
|||||||
insert_format = "Values";
|
insert_format = "Values";
|
||||||
insert_format_max_block_size = config().getInt("insert_format_max_block_size", context.getSettingsRef().max_insert_block_size);
|
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();
|
connect();
|
||||||
|
|
||||||
if (is_interactive)
|
if (is_interactive)
|
||||||
@ -850,11 +854,11 @@ private:
|
|||||||
/// Формат может быть указан в запросе.
|
/// Формат может быть указан в запросе.
|
||||||
if (ASTQueryWithOutput * query_with_output = dynamic_cast<ASTQueryWithOutput *>(&*parsed_query))
|
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)
|
if (has_vertical_output_suffix)
|
||||||
throw Exception("Output format already specified", ErrorCodes::CLIENT_OUTPUT_FORMAT_SPECIFIED);
|
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;
|
current_format = id->name;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -906,7 +910,7 @@ private:
|
|||||||
|
|
||||||
void writeProgress()
|
void writeProgress()
|
||||||
{
|
{
|
||||||
if (!is_interactive)
|
if (!need_render_progress)
|
||||||
return;
|
return;
|
||||||
|
|
||||||
static size_t increment = 0;
|
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")
|
("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)")
|
("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)")
|
||||||
("stacktrace", "print stack traces of exceptions")
|
("stacktrace", "print stack traces of exceptions")
|
||||||
|
("progress", "print progress even in non-interactive mode")
|
||||||
APPLY_FOR_SETTINGS(DECLARE_SETTING)
|
APPLY_FOR_SETTINGS(DECLARE_SETTING)
|
||||||
APPLY_FOR_LIMITS(DECLARE_LIMIT)
|
APPLY_FOR_LIMITS(DECLARE_LIMIT)
|
||||||
;
|
;
|
||||||
@ -1171,6 +1176,8 @@ public:
|
|||||||
config().setBool("vertical", true);
|
config().setBool("vertical", true);
|
||||||
if (options.count("stacktrace"))
|
if (options.count("stacktrace"))
|
||||||
config().setBool("stacktrace", true);
|
config().setBool("stacktrace", true);
|
||||||
|
if (options.count("progress"))
|
||||||
|
config().setBool("progress", true);
|
||||||
if (options.count("time"))
|
if (options.count("time"))
|
||||||
print_time_to_stderr = true;
|
print_time_to_stderr = true;
|
||||||
}
|
}
|
||||||
|
@ -3,7 +3,7 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
ParallelReplicas::ParallelReplicas(Connection * connection_, Settings * settings_, ThrottlerPtr throttler_)
|
ParallelReplicas::ParallelReplicas(Connection * connection_, const Settings * settings_, ThrottlerPtr throttler_)
|
||||||
: settings(settings_), throttler(throttler_),
|
: settings(settings_), throttler(throttler_),
|
||||||
active_replica_count(1),
|
active_replica_count(1),
|
||||||
supports_parallel_execution(false)
|
supports_parallel_execution(false)
|
||||||
@ -11,7 +11,7 @@ ParallelReplicas::ParallelReplicas(Connection * connection_, Settings * settings
|
|||||||
registerReplica(connection_);
|
registerReplica(connection_);
|
||||||
}
|
}
|
||||||
|
|
||||||
ParallelReplicas::ParallelReplicas(IConnectionPool * pool_, Settings * settings_, ThrottlerPtr throttler_)
|
ParallelReplicas::ParallelReplicas(IConnectionPool * pool_, const Settings * settings_, ThrottlerPtr throttler_)
|
||||||
: settings(settings_), throttler(throttler_)
|
: settings(settings_), throttler(throttler_)
|
||||||
{
|
{
|
||||||
if (pool_ == nullptr)
|
if (pool_ == nullptr)
|
||||||
@ -37,7 +37,8 @@ ParallelReplicas::ParallelReplicas(IConnectionPool * pool_, Settings * settings_
|
|||||||
supports_parallel_execution = false;
|
supports_parallel_execution = false;
|
||||||
|
|
||||||
pool_entry = pool_->get(settings);
|
pool_entry = pool_->get(settings);
|
||||||
registerReplica(&*pool_entry);
|
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/DataTypeFixedString.h>
|
||||||
|
#include <DB/DataTypes/DataTypeArray.h>
|
||||||
|
|
||||||
#include <DB/Columns/ColumnString.h>
|
#include <DB/Columns/ColumnString.h>
|
||||||
#include <DB/Columns/ColumnConst.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)
|
if (!data_type)
|
||||||
throw Exception("No data type specified for ColumnConstArray", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("No data type specified for ColumnConstArray", ErrorCodes::LOGICAL_ERROR);
|
||||||
@ -70,7 +71,8 @@ template <> ColumnPtr ColumnConst<Array>::convertToFullColumn() const
|
|||||||
if (!type)
|
if (!type)
|
||||||
throw Exception("Non-array data type specified for ColumnConstArray", ErrorCodes::LOGICAL_ERROR);
|
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();
|
ColumnPtr nested_column = type->getNestedType()->createColumn();
|
||||||
|
|
||||||
ColumnArray * res = new ColumnArray(nested_column);
|
ColumnArray * res = new ColumnArray(nested_column);
|
||||||
@ -81,11 +83,27 @@ template <> ColumnPtr ColumnConst<Array>::convertToFullColumn() const
|
|||||||
{
|
{
|
||||||
offsets[i] = (i + 1) * array_size;
|
offsets[i] = (i + 1) * array_size;
|
||||||
for (size_t j = 0; j < array_size; ++j)
|
for (size_t j = 0; j < array_size; ++j)
|
||||||
nested_column->insert(data[j]);
|
nested_column->insert(array[j]);
|
||||||
}
|
}
|
||||||
|
|
||||||
return res;
|
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)
|
void tryLogCurrentException(Poco::Logger * logger)
|
||||||
{
|
{
|
||||||
|
try
|
||||||
|
{
|
||||||
|
LOG_ERROR(logger, getCurrentExceptionMessage(true));
|
||||||
|
}
|
||||||
|
catch (...)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
std::string getCurrentExceptionMessage(bool with_stacktrace)
|
||||||
|
{
|
||||||
|
std::stringstream stream;
|
||||||
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
throw;
|
throw;
|
||||||
@ -67,8 +80,10 @@ void tryLogCurrentException(Poco::Logger * logger)
|
|||||||
{
|
{
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
LOG_ERROR(logger, "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what()
|
stream << "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what();
|
||||||
<< ", Stack trace:\n\n" << e.getStackTrace().toString());
|
|
||||||
|
if (with_stacktrace)
|
||||||
|
stream << ", Stack trace:\n\n" << e.getStackTrace().toString();
|
||||||
}
|
}
|
||||||
catch (...) {}
|
catch (...) {}
|
||||||
}
|
}
|
||||||
@ -76,8 +91,8 @@ void tryLogCurrentException(Poco::Logger * logger)
|
|||||||
{
|
{
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
LOG_ERROR(logger, "Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
|
stream << "Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
|
||||||
<< ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what());
|
<< ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what();
|
||||||
}
|
}
|
||||||
catch (...) {}
|
catch (...) {}
|
||||||
}
|
}
|
||||||
@ -91,7 +106,7 @@ void tryLogCurrentException(Poco::Logger * logger)
|
|||||||
if (status)
|
if (status)
|
||||||
name += " (demangling status: " + toString(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 (...) {}
|
catch (...) {}
|
||||||
}
|
}
|
||||||
@ -105,10 +120,12 @@ void tryLogCurrentException(Poco::Logger * logger)
|
|||||||
if (status)
|
if (status)
|
||||||
name += " (demangling status: " + toString(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 (...) {}
|
catch (...) {}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
return stream.str();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
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);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
}
|
@ -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)
|
void assertEOF(ReadBuffer & buf)
|
||||||
{
|
{
|
||||||
if (!buf.eof())
|
if (!buf.eof())
|
||||||
|
@ -557,14 +557,21 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
|
|||||||
for (size_t i = 0; i < aggregates_size; ++i)
|
for (size_t i = 0; i < aggregates_size; ++i)
|
||||||
aggregate_columns[i].resize(aggregates[i].arguments.size());
|
aggregate_columns[i].resize(aggregates[i].arguments.size());
|
||||||
|
|
||||||
|
/** Константные столбцы не поддерживаются напрямую при агрегации.
|
||||||
|
* Чтобы они всё-равно работали, материализуем их.
|
||||||
|
*/
|
||||||
|
Columns materialized_columns;
|
||||||
|
|
||||||
/// Запоминаем столбцы, с которыми будем работать
|
/// Запоминаем столбцы, с которыми будем работать
|
||||||
for (size_t i = 0; i < keys_size; ++i)
|
for (size_t i = 0; i < keys_size; ++i)
|
||||||
{
|
{
|
||||||
key_columns[i] = block.getByPosition(keys[i]).column;
|
key_columns[i] = block.getByPosition(keys[i]).column;
|
||||||
|
|
||||||
if (key_columns[i]->isConst())
|
if (const IColumnConst * column_const = dynamic_cast<const IColumnConst *>(key_columns[i]))
|
||||||
throw Exception("Constants are not allowed as GROUP BY keys"
|
{
|
||||||
" (but all of them must be eliminated in ExpressionAnalyzer)", ErrorCodes::ILLEGAL_COLUMN);
|
materialized_columns.push_back(column_const->convertToFullColumn());
|
||||||
|
key_columns[i] = materialized_columns.back().get();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
for (size_t i = 0; i < aggregates_size; ++i)
|
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;
|
aggregate_columns[i][j] = block.getByPosition(aggregates[i].arguments[j]).column;
|
||||||
|
|
||||||
/** Агрегатные функции рассчитывают, что в них передаются полноценные столбцы.
|
if (const IColumnConst * column_const = dynamic_cast<const IColumnConst *>(aggregate_columns[i][j]))
|
||||||
* Поэтому, стобцы-константы не разрешены в качестве аргументов агрегатных функций.
|
{
|
||||||
*/
|
materialized_columns.push_back(column_const->convertToFullColumn());
|
||||||
if (aggregate_columns[i][j]->isConst())
|
aggregate_columns[i][j] = materialized_columns.back().get();
|
||||||
throw Exception("Constants are not allowed as arguments of aggregate functions", ErrorCodes::ILLEGAL_COLUMN);
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -27,6 +27,7 @@
|
|||||||
#include <DB/Interpreters/Cluster.h>
|
#include <DB/Interpreters/Cluster.h>
|
||||||
#include <DB/Interpreters/InterserverIOHandler.h>
|
#include <DB/Interpreters/InterserverIOHandler.h>
|
||||||
#include <DB/Interpreters/Compiler.h>
|
#include <DB/Interpreters/Compiler.h>
|
||||||
|
#include <DB/Interpreters/QueryLog.h>
|
||||||
#include <DB/Interpreters/Context.h>
|
#include <DB/Interpreters/Context.h>
|
||||||
#include <DB/IO/ReadBufferFromFile.h>
|
#include <DB/IO/ReadBufferFromFile.h>
|
||||||
#include <DB/IO/WriteBufferFromString.h>
|
#include <DB/IO/WriteBufferFromString.h>
|
||||||
@ -86,6 +87,7 @@ struct ContextShared
|
|||||||
BackgroundProcessingPoolPtr background_pool; /// Пул потоков для фоновой работы, выполняемой таблицами.
|
BackgroundProcessingPoolPtr background_pool; /// Пул потоков для фоновой работы, выполняемой таблицами.
|
||||||
Macros macros; /// Подстановки из конфига.
|
Macros macros; /// Подстановки из конфига.
|
||||||
std::unique_ptr<Compiler> compiler; /// Для динамической компиляции частей запроса, при необходимости.
|
std::unique_ptr<Compiler> compiler; /// Для динамической компиляции частей запроса, при необходимости.
|
||||||
|
std::unique_ptr<QueryLog> query_log; /// Для логгирования запросов.
|
||||||
mutable std::unique_ptr<CompressionMethodSelector> compression_method_selector; /// Правила для выбора метода сжатия в зависимости от размера куска.
|
mutable std::unique_ptr<CompressionMethodSelector> compression_method_selector; /// Правила для выбора метода сжатия в зависимости от размера куска.
|
||||||
|
|
||||||
/// Кластеры для distributed таблиц
|
/// Кластеры для 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
|
CompressionMethod Context::chooseCompressionMethod(size_t part_size, double part_size_ratio) const
|
||||||
{
|
{
|
||||||
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
|
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
|
||||||
|
@ -1399,6 +1399,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl
|
|||||||
else if (ASTLiteral * node = typeid_cast<ASTLiteral *>(&*ast))
|
else if (ASTLiteral * node = typeid_cast<ASTLiteral *>(&*ast))
|
||||||
{
|
{
|
||||||
DataTypePtr type = apply_visitor(FieldToDataType(), node->value);
|
DataTypePtr type = apply_visitor(FieldToDataType(), node->value);
|
||||||
|
|
||||||
ColumnWithNameAndType column;
|
ColumnWithNameAndType column;
|
||||||
column.column = type->createConstColumn(1, node->value);
|
column.column = type->createConstColumn(1, node->value);
|
||||||
column.type = type;
|
column.type = type;
|
||||||
|
@ -81,6 +81,10 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error)
|
|||||||
{
|
{
|
||||||
const std::lock_guard<std::mutex> lock{dictionaries_mutex};
|
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);
|
const auto dict_it = dictionaries.find(name);
|
||||||
if (dict_it->second.dict)
|
if (dict_it->second.dict)
|
||||||
dict_it->second.dict->set(dict_ptr.release());
|
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
|
/// erase stored exception on success
|
||||||
dict_it->second.exception = std::exception_ptr{};
|
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);
|
recreated_failed_dictionaries.push_back(name);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
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);
|
__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);
|
__sync_fetch_and_add(&used.errors, 1);
|
||||||
}
|
}
|
||||||
@ -181,7 +181,7 @@ void QuotaForIntervals::addQuery(time_t current_time)
|
|||||||
it->second.addQuery(current_time, name);
|
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)
|
for (Container::reverse_iterator it = cont.rbegin(); it != cont.rend(); ++it)
|
||||||
it->second.addError(current_time, name);
|
it->second.addError(current_time, name);
|
||||||
|
@ -1,10 +1,12 @@
|
|||||||
#include <DB/Common/ProfileEvents.h>
|
#include <DB/Common/ProfileEvents.h>
|
||||||
|
#include <DB/Common/formatReadable.h>
|
||||||
|
|
||||||
#include <DB/IO/ConcatReadBuffer.h>
|
#include <DB/IO/ConcatReadBuffer.h>
|
||||||
|
|
||||||
#include <DB/DataStreams/BlockIO.h>
|
#include <DB/DataStreams/BlockIO.h>
|
||||||
#include <DB/DataStreams/FormatFactory.h>
|
#include <DB/DataStreams/FormatFactory.h>
|
||||||
#include <DB/DataStreams/copyData.h>
|
#include <DB/DataStreams/copyData.h>
|
||||||
|
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||||
|
|
||||||
#include <DB/Parsers/ASTInsertQuery.h>
|
#include <DB/Parsers/ASTInsertQuery.h>
|
||||||
#include <DB/Parsers/ASTShowProcesslistQuery.h>
|
#include <DB/Parsers/ASTShowProcesslistQuery.h>
|
||||||
@ -15,6 +17,7 @@
|
|||||||
#include <DB/Interpreters/Quota.h>
|
#include <DB/Interpreters/Quota.h>
|
||||||
#include <DB/Interpreters/InterpreterFactory.h>
|
#include <DB/Interpreters/InterpreterFactory.h>
|
||||||
#include <DB/Interpreters/ProcessList.h>
|
#include <DB/Interpreters/ProcessList.h>
|
||||||
|
#include <DB/Interpreters/QueryLog.h>
|
||||||
#include <DB/Interpreters/executeQuery.h>
|
#include <DB/Interpreters/executeQuery.h>
|
||||||
|
|
||||||
|
|
||||||
@ -31,6 +34,7 @@ static void checkLimits(const IAST & ast, const Limits & limits)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/// Логгировать запрос в обычный лог (не в таблицу).
|
||||||
static void logQuery(const String & query, const Context & context)
|
static void logQuery(const String & query, const Context & context)
|
||||||
{
|
{
|
||||||
String logged_query = query;
|
String logged_query = query;
|
||||||
@ -39,6 +43,53 @@ static void logQuery(const String & query, const Context & context)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
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(
|
static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||||
IParser::Pos begin,
|
IParser::Pos begin,
|
||||||
IParser::Pos end,
|
IParser::Pos end,
|
||||||
@ -47,6 +98,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
QueryProcessingStage::Enum stage)
|
QueryProcessingStage::Enum stage)
|
||||||
{
|
{
|
||||||
ProfileEvents::increment(ProfileEvents::Query);
|
ProfileEvents::increment(ProfileEvents::Query);
|
||||||
|
time_t current_time = time(0);
|
||||||
|
|
||||||
ParserQuery parser;
|
ParserQuery parser;
|
||||||
ASTPtr ast;
|
ASTPtr ast;
|
||||||
@ -67,58 +119,176 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
catch (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
/// Всё равно логгируем запрос.
|
/// Всё равно логгируем запрос.
|
||||||
logQuery(String(begin, begin + std::min(end - begin, static_cast<ptrdiff_t>(max_query_size))), context);
|
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;
|
throw;
|
||||||
}
|
}
|
||||||
|
|
||||||
String query(begin, query_size);
|
String query(begin, query_size);
|
||||||
|
|
||||||
if (!internal)
|
|
||||||
logQuery(query, context);
|
|
||||||
|
|
||||||
/// Проверка ограничений.
|
|
||||||
checkLimits(*ast, context.getSettingsRef().limits);
|
|
||||||
|
|
||||||
QuotaForIntervals & quota = context.getQuota();
|
|
||||||
time_t current_time = time(0);
|
|
||||||
|
|
||||||
quota.checkExceeded(current_time);
|
|
||||||
|
|
||||||
/// Положим запрос в список процессов. Но запрос SHOW PROCESSLIST класть не будем.
|
|
||||||
ProcessList::EntryPtr process_list_entry;
|
|
||||||
if (!internal && nullptr == typeid_cast<const ASTShowProcesslistQuery *>(&*ast))
|
|
||||||
{
|
|
||||||
const Settings & settings = context.getSettingsRef();
|
|
||||||
|
|
||||||
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());
|
|
||||||
}
|
|
||||||
|
|
||||||
BlockIO res;
|
BlockIO res;
|
||||||
|
|
||||||
try
|
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);
|
auto interpreter = InterpreterFactory::get(ast, context, stage);
|
||||||
res = interpreter->execute();
|
res = interpreter->execute();
|
||||||
|
|
||||||
/// Держим элемент списка процессов до конца обработки запроса.
|
/// Держим элемент списка процессов до конца обработки запроса.
|
||||||
res.process_list_entry = process_list_entry;
|
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 (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
quota.addError(current_time);
|
onExceptionBeforeStart(query, context, current_time);
|
||||||
throw;
|
throw;
|
||||||
}
|
}
|
||||||
|
|
||||||
quota.addQuery(current_time);
|
|
||||||
|
|
||||||
return std::make_tuple(ast, res);
|
return std::make_tuple(ast, res);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -174,51 +344,61 @@ void executeQuery(
|
|||||||
|
|
||||||
std::tie(ast, streams) = executeQueryImpl(begin, end, context, internal, stage);
|
std::tie(ast, streams) = executeQueryImpl(begin, end, context, internal, stage);
|
||||||
|
|
||||||
if (streams.out)
|
try
|
||||||
{
|
{
|
||||||
const ASTInsertQuery * ast_insert_query = dynamic_cast<const ASTInsertQuery *>(ast.get());
|
if (streams.out)
|
||||||
|
{
|
||||||
|
const ASTInsertQuery * ast_insert_query = dynamic_cast<const ASTInsertQuery *>(ast.get());
|
||||||
|
|
||||||
if (!ast_insert_query)
|
if (!ast_insert_query)
|
||||||
throw Exception("Logical error: query requires data to insert, but it is not INSERT query", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Logical error: query requires data to insert, but it is not INSERT query", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
String format = ast_insert_query->format;
|
String format = ast_insert_query->format;
|
||||||
if (format.empty())
|
if (format.empty())
|
||||||
format = "Values";
|
format = "Values";
|
||||||
|
|
||||||
/// Данные могут содержаться в распарсенной (ast_insert_query.data) и ещё не распарсенной (istr) части запроса.
|
/// Данные могут содержаться в распарсенной (ast_insert_query.data) и ещё не распарсенной (istr) части запроса.
|
||||||
|
|
||||||
ConcatReadBuffer::ReadBuffers buffers;
|
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);
|
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)
|
if (ast_insert_query->data)
|
||||||
buffers.push_back(&buf1);
|
buffers.push_back(&buf1);
|
||||||
buffers.push_back(&istr);
|
buffers.push_back(&istr);
|
||||||
|
|
||||||
/** NOTE Нельзя читать из istr до того, как прочтём всё между ast_insert_query.data и ast_insert_query.end.
|
/** NOTE Нельзя читать из istr до того, как прочтём всё между ast_insert_query.data и ast_insert_query.end.
|
||||||
* - потому что query.data может ссылаться на кусок памяти, использующийся в качестве буфера в istr.
|
* - потому что query.data может ссылаться на кусок памяти, использующийся в качестве буфера в istr.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
ConcatReadBuffer data_istr(buffers);
|
ConcatReadBuffer data_istr(buffers);
|
||||||
|
|
||||||
BlockInputStreamPtr in{
|
BlockInputStreamPtr in{
|
||||||
context.getFormatFactory().getInput(
|
context.getFormatFactory().getInput(
|
||||||
format, data_istr, streams.out_sample, context.getSettings().max_insert_block_size)};
|
format, data_istr, streams.out_sample, context.getSettings().max_insert_block_size)};
|
||||||
|
|
||||||
copyData(*in, *streams.out);
|
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 (...)
|
||||||
|
{
|
||||||
|
streams.onException();
|
||||||
|
throw;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (streams.in)
|
streams.onFinish();
|
||||||
{
|
|
||||||
const ASTQueryWithOutput * ast_query_with_output = dynamic_cast<const ASTQueryWithOutput *>(ast.get());
|
|
||||||
|
|
||||||
String format_name = ast_query_with_output && ast_query_with_output->format
|
|
||||||
? typeid_cast<const ASTIdentifier &>(*ast_query_with_output->format).name
|
|
||||||
: context.getDefaultFormat();
|
|
||||||
|
|
||||||
BlockOutputStreamPtr out = context.getFormatFactory().getOutput(format_name, ostr, streams.in_sample);
|
|
||||||
|
|
||||||
copyData(*streams.in, *out);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
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;
|
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)
|
bool ParserLiteral::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
|
||||||
{
|
{
|
||||||
ParserNull null_p;
|
ParserNull null_p;
|
||||||
@ -450,6 +503,7 @@ bool ParserExpressionElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos &
|
|||||||
ParserParenthesisExpression paren_p;
|
ParserParenthesisExpression paren_p;
|
||||||
ParserSubquery subquery_p;
|
ParserSubquery subquery_p;
|
||||||
ParserArray array_p;
|
ParserArray array_p;
|
||||||
|
ParserArrayOfLiterals array_lite_p;
|
||||||
ParserLiteral lit_p;
|
ParserLiteral lit_p;
|
||||||
ParserFunction fun_p;
|
ParserFunction fun_p;
|
||||||
ParserCompoundIdentifier id_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))
|
if (paren_p.parse(pos, end, node, max_parsed_pos, expected))
|
||||||
return true;
|
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))
|
if (array_p.parse(pos, end, node, max_parsed_pos, expected))
|
||||||
return true;
|
return true;
|
||||||
|
|
||||||
|
@ -8,7 +8,6 @@
|
|||||||
#include <DB/Parsers/ParserSetQuery.h>
|
#include <DB/Parsers/ParserSetQuery.h>
|
||||||
#include <DB/Parsers/ParserSelectQuery.h>
|
#include <DB/Parsers/ParserSelectQuery.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -296,6 +295,8 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
|||||||
ws.ignore(pos, end);
|
ws.ignore(pos, end);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
bool has_format = false;
|
||||||
|
|
||||||
/// FORMAT format_name
|
/// FORMAT format_name
|
||||||
if (s_format.ignore(pos, end, max_parsed_pos, expected))
|
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;
|
typeid_cast<ASTIdentifier &>(*select_query->format).kind = ASTIdentifier::Format;
|
||||||
|
|
||||||
ws.ignore(pos, end);
|
ws.ignore(pos, end);
|
||||||
|
has_format = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
// UNION ALL select query
|
// 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 (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;
|
ParserSelectQuery select_p;
|
||||||
if (!select_p.parse(pos, end, select_query->next_union_all, max_parsed_pos, expected))
|
if (!select_p.parse(pos, end, select_query->next_union_all, max_parsed_pos, expected))
|
||||||
return false;
|
return false;
|
||||||
|
@ -136,41 +136,24 @@ void HTTPHandler::processQuery(Poco::Net::HTTPServerRequest & request, Poco::Net
|
|||||||
if (readonly)
|
if (readonly)
|
||||||
context.getSettingsRef().limits.readonly = true;
|
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);
|
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.
|
/// Если не было эксепшена и данные ещё не отправлены - отправляются HTTP заголовки с кодом 200.
|
||||||
used_output.out->finalize();
|
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,
|
Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response,
|
||||||
Output & used_output)
|
Output & used_output)
|
||||||
{
|
{
|
||||||
@ -191,7 +174,7 @@ void HTTPHandler::trySendExceptionToClient(std::stringstream & s,
|
|||||||
if (!response.sent() && !used_output.out_maybe_compressed)
|
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)
|
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();
|
used_output.out->position() = used_output.out->buffer().begin();
|
||||||
}
|
}
|
||||||
|
|
||||||
std::string exception_message = s.str();
|
writeString(s, *used_output.out_maybe_compressed);
|
||||||
writeString(exception_message, *used_output.out_maybe_compressed);
|
|
||||||
writeChar('\n', *used_output.out_maybe_compressed);
|
writeChar('\n', *used_output.out_maybe_compressed);
|
||||||
used_output.out_maybe_compressed->next();
|
used_output.out_maybe_compressed->next();
|
||||||
used_output.out->finalize();
|
used_output.out->finalize();
|
||||||
@ -245,35 +227,10 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
|
|||||||
processQuery(request, response, used_output);
|
processQuery(request, response, used_output);
|
||||||
LOG_INFO(log, "Done processing query");
|
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 (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
std::stringstream s;
|
tryLogCurrentException(log);
|
||||||
s << "Code: " << ErrorCodes::UNKNOWN_EXCEPTION << ". Unknown exception.";
|
trySendExceptionToClient(getCurrentExceptionMessage(true), request, response, used_output);
|
||||||
LOG_ERROR(log, s.str());
|
|
||||||
trySendExceptionToClient(s, request, response, used_output);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -26,7 +26,7 @@ public:
|
|||||||
|
|
||||||
void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response);
|
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,
|
Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response,
|
||||||
Output & used_output);
|
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.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);
|
OLAP::QueryParseResult olap_query = server.olap_parser->parse(request_istream);
|
||||||
|
|
||||||
std::string clickhouse_query;
|
std::string clickhouse_query;
|
||||||
@ -66,31 +69,6 @@ void OLAPHTTPHandler::processQuery(Poco::Net::HTTPServerRequest & request, Poco:
|
|||||||
executeQuery(in, out, context, query_plan);
|
executeQuery(in, out, context, query_plan);
|
||||||
watch.stop();
|
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.
|
/// Если не было эксепшена и данные ещё не отправлены - отправляются HTTP заголовки с кодом 200.
|
||||||
out.finalize();
|
out.finalize();
|
||||||
}
|
}
|
||||||
|
@ -533,7 +533,12 @@ int Server::main(const std::vector<std::string> & args)
|
|||||||
LOG_DEBUG(log, "Loaded metadata.");
|
LOG_DEBUG(log, "Loaded metadata.");
|
||||||
|
|
||||||
/// Создаём системные таблицы.
|
/// Создаём системные таблицы.
|
||||||
global_context->addDatabase("system");
|
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", "one", StorageSystemOne::create("one"));
|
||||||
global_context->addTable("system", "numbers", StorageSystemNumbers::create("numbers"));
|
global_context->addTable("system", "numbers", StorageSystemNumbers::create("numbers"));
|
||||||
@ -546,7 +551,7 @@ int Server::main(const std::vector<std::string> & args)
|
|||||||
global_context->addTable("system", "events", StorageSystemEvents::create("events"));
|
global_context->addTable("system", "events", StorageSystemEvents::create("events"));
|
||||||
global_context->addTable("system", "merges", StorageSystemMerges::create("merges"));
|
global_context->addTable("system", "merges", StorageSystemMerges::create("merges"));
|
||||||
global_context->addTable("system", "replicas", StorageSystemReplicas::create("replicas"));
|
global_context->addTable("system", "replicas", StorageSystemReplicas::create("replicas"));
|
||||||
global_context->addTable("system", "dictionaries", StorageSystemDictionaries::create("dictionaries"));
|
global_context->addTable("system", "dictionaries", StorageSystemDictionaries::create("dictionaries"));
|
||||||
global_context->addTable("system", "columns", StorageSystemColumns::create("columns"));
|
global_context->addTable("system", "columns", StorageSystemColumns::create("columns"));
|
||||||
global_context->addTable("system", "functions", StorageSystemFunctions::create("functions"));
|
global_context->addTable("system", "functions", StorageSystemFunctions::create("functions"));
|
||||||
global_context->addTable("system", "clusters", StorageSystemClusters::create("clusters", *global_context));
|
global_context->addTable("system", "clusters", StorageSystemClusters::create("clusters", *global_context));
|
||||||
|
@ -111,6 +111,7 @@ void TCPHandler::runImpl()
|
|||||||
{
|
{
|
||||||
/// Восстанавливаем контекст запроса.
|
/// Восстанавливаем контекст запроса.
|
||||||
query_context = connection_context;
|
query_context = connection_context;
|
||||||
|
query_context.setInterface(Context::Interface::TCP);
|
||||||
|
|
||||||
/** Если Query - обрабатываем. Если Ping или Cancel - возвращаемся в начало.
|
/** Если Query - обрабатываем. Если Ping или Cancel - возвращаемся в начало.
|
||||||
* Могут прийти настройки на отдельный запрос, которые модифицируют query_context.
|
* Могут прийти настройки на отдельный запрос, которые модифицируют query_context.
|
||||||
@ -148,8 +149,7 @@ void TCPHandler::runImpl()
|
|||||||
}
|
}
|
||||||
catch (const Exception & e)
|
catch (const Exception & e)
|
||||||
{
|
{
|
||||||
LOG_ERROR(log, "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what()
|
state.io.onException();
|
||||||
<< ", Stack trace:\n\n" << e.getStackTrace().toString());
|
|
||||||
exception = e.clone();
|
exception = e.clone();
|
||||||
|
|
||||||
if (e.code() == ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT)
|
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()
|
state.io.onException();
|
||||||
<< ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what());
|
|
||||||
exception = new Exception(e.displayText(), ErrorCodes::POCO_EXCEPTION);
|
exception = new Exception(e.displayText(), ErrorCodes::POCO_EXCEPTION);
|
||||||
}
|
}
|
||||||
catch (const Poco::Exception & e)
|
catch (const Poco::Exception & e)
|
||||||
{
|
{
|
||||||
LOG_ERROR(log, "Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
|
state.io.onException();
|
||||||
<< ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what());
|
|
||||||
exception = new Exception(e.displayText(), ErrorCodes::POCO_EXCEPTION);
|
exception = new Exception(e.displayText(), ErrorCodes::POCO_EXCEPTION);
|
||||||
}
|
}
|
||||||
catch (const std::exception & e)
|
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);
|
exception = new Exception(e.what(), ErrorCodes::STD_EXCEPTION);
|
||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
LOG_ERROR(log, "Unknown exception. Code: " << ErrorCodes::UNKNOWN_EXCEPTION);
|
state.io.onException();
|
||||||
exception = new Exception("Unknown exception", ErrorCodes::UNKNOWN_EXCEPTION);
|
exception = new Exception("Unknown exception", ErrorCodes::UNKNOWN_EXCEPTION);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -285,11 +283,6 @@ void TCPHandler::processOrdinaryQuery()
|
|||||||
AsynchronousBlockInputStream async_in(state.io.in);
|
AsynchronousBlockInputStream async_in(state.io.in);
|
||||||
async_in.readPrefix();
|
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)
|
while (true)
|
||||||
{
|
{
|
||||||
Block block;
|
Block block;
|
||||||
@ -341,9 +334,7 @@ void TCPHandler::processOrdinaryQuery()
|
|||||||
}
|
}
|
||||||
|
|
||||||
async_in.readSuffix();
|
async_in.readSuffix();
|
||||||
|
state.io.onFinish();
|
||||||
watch.stop();
|
|
||||||
logProfileInfo(watch, *state.io.in);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -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()
|
void TCPHandler::receiveHello()
|
||||||
{
|
{
|
||||||
/// Получить hello пакет.
|
/// Получить hello пакет.
|
||||||
|
@ -130,9 +130,6 @@ private:
|
|||||||
|
|
||||||
/// Эта функция вызывается из разных потоков.
|
/// Эта функция вызывается из разных потоков.
|
||||||
void updateProgress(const Progress & value);
|
void updateProgress(const Progress & value);
|
||||||
|
|
||||||
/// Вывести информацию о скорости выполнения SELECT запроса.
|
|
||||||
void logProfileInfo(Stopwatch & watch, IBlockInputStream & in);
|
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
@ -11,7 +11,6 @@
|
|||||||
#include <DB/IO/CompressedReadBuffer.h>
|
#include <DB/IO/CompressedReadBuffer.h>
|
||||||
#include <DB/IO/HashingReadBuffer.h>
|
#include <DB/IO/HashingReadBuffer.h>
|
||||||
#include <DB/Columns/ColumnsNumber.h>
|
#include <DB/Columns/ColumnsNumber.h>
|
||||||
#include <statdaemons/ext/scope_guard.hpp>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -116,12 +115,6 @@ struct Stream
|
|||||||
readIntBinary(mrk_mark.offset_in_compressed_file, mrk_hashing_buf);
|
readIntBinary(mrk_mark.offset_in_compressed_file, mrk_hashing_buf);
|
||||||
readIntBinary(mrk_mark.offset_in_decompressed_block, mrk_hashing_buf);
|
readIntBinary(mrk_mark.offset_in_decompressed_block, mrk_hashing_buf);
|
||||||
|
|
||||||
/// На всякий случай, сохраним смещение в файле и размер предыдущего блока.
|
|
||||||
SCOPE_EXIT(
|
|
||||||
prev_offset_in_compressed_file = mrk_mark.offset_in_compressed_file;
|
|
||||||
prev_buffer_size = uncompressed_hashing_buf.buffer().size();
|
|
||||||
);
|
|
||||||
|
|
||||||
bool has_alternative_mark = false;
|
bool has_alternative_mark = false;
|
||||||
MarkInCompressedFile alternative_data_mark;
|
MarkInCompressedFile alternative_data_mark;
|
||||||
MarkInCompressedFile data_mark;
|
MarkInCompressedFile data_mark;
|
||||||
@ -145,18 +138,6 @@ struct Stream
|
|||||||
if (uncompressed_hashing_buf.eof())
|
if (uncompressed_hashing_buf.eof())
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
else if (uncompressed_hashing_buf.offset() == 0)
|
|
||||||
{
|
|
||||||
/// Восстановим засечку на конец предыдущего блока по сохраненным данным
|
|
||||||
has_alternative_mark = true;
|
|
||||||
alternative_data_mark.offset_in_compressed_file = prev_offset_in_compressed_file;
|
|
||||||
alternative_data_mark.offset_in_decompressed_block = prev_buffer_size;
|
|
||||||
|
|
||||||
if (mrk_mark == alternative_data_mark)
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
std::cout << "mrk_mark " << mrk_mark.offset_in_compressed_file << ' ' << mrk_mark.offset_in_decompressed_block << std::endl;
|
|
||||||
|
|
||||||
data_mark.offset_in_compressed_file = compressed_hashing_buf.count() - uncompressing_buf.getSizeCompressed();
|
data_mark.offset_in_compressed_file = compressed_hashing_buf.count() - uncompressing_buf.getSizeCompressed();
|
||||||
data_mark.offset_in_decompressed_block = uncompressed_hashing_buf.offset();
|
data_mark.offset_in_decompressed_block = uncompressed_hashing_buf.offset();
|
||||||
@ -180,10 +161,6 @@ struct Stream
|
|||||||
checksums.files[name + ".mrk"] = MergeTreeData::DataPart::Checksums::Checksum(
|
checksums.files[name + ".mrk"] = MergeTreeData::DataPart::Checksums::Checksum(
|
||||||
mrk_hashing_buf.count(), mrk_hashing_buf.getHash());
|
mrk_hashing_buf.count(), mrk_hashing_buf.getHash());
|
||||||
}
|
}
|
||||||
|
|
||||||
private:
|
|
||||||
size_t prev_offset_in_compressed_file{};
|
|
||||||
size_t prev_buffer_size{};
|
|
||||||
};
|
};
|
||||||
|
|
||||||
/// Возвращает количество строк. Добавляет в checksums чексуммы всех файлов столбца.
|
/// Возвращает количество строк. Добавляет в checksums чексуммы всех файлов столбца.
|
||||||
|
@ -8,6 +8,136 @@
|
|||||||
namespace DB
|
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_)
|
PKCondition::PKCondition(ASTPtr query, const Context & context_, const NamesAndTypesList & all_columns, const SortDescription & sort_descr_)
|
||||||
: sort_descr(sort_descr_)
|
: sort_descr(sort_descr_)
|
||||||
{
|
{
|
||||||
|
@ -545,7 +545,8 @@ bool StorageChunkMerger::MergeTask::mergeChunks(const Storages & chunks)
|
|||||||
{
|
{
|
||||||
LOG_INFO(log, "Shutdown requested while merging chunks.");
|
LOG_INFO(log, "Shutdown requested while merging chunks.");
|
||||||
output->writeSuffix();
|
output->writeSuffix();
|
||||||
new_storage.removeReference(); /// После этого временные данные удалятся.
|
output = nullptr;
|
||||||
|
executeQuery("DROP TABLE IF EXISTS " + new_table_full_name, context, true);
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -575,14 +576,16 @@ bool StorageChunkMerger::MergeTask::mergeChunks(const Storages & chunks)
|
|||||||
/// Отцепляем исходную таблицу. Ее данные и метаданные остаются на диске.
|
/// Отцепляем исходную таблицу. Ее данные и метаданные остаются на диске.
|
||||||
tables_to_drop.push_back(context.detachTable(chunk_merger.source_database, src_name));
|
tables_to_drop.push_back(context.detachTable(chunk_merger.source_database, src_name));
|
||||||
|
|
||||||
/// Создаем на ее месте ChunkRef. Это возможно только потому что у ChunkRef нет ни, ни метаданных.
|
/// Создаем на ее месте ChunkRef. Это возможно только потому что у ChunkRef нет ни данных, ни метаданных.
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
context.addTable(chunk_merger.source_database, src_name, StorageChunkRef::create(src_name, context, chunk_merger.source_database, new_table_name, false));
|
context.addTable(chunk_merger.source_database, src_name,
|
||||||
|
StorageChunkRef::create(src_name, context, chunk_merger.source_database, new_table_name, false));
|
||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
LOG_ERROR(log, "Chunk " + src_name + " was removed but not replaced. Its data is stored in table " << new_table_name << ". You may need to resolve this manually.");
|
LOG_ERROR(log, "Chunk " + src_name + " was removed but not replaced. Its data is stored in table "
|
||||||
|
<< new_table_name << ". You may need to resolve this manually.");
|
||||||
|
|
||||||
throw;
|
throw;
|
||||||
}
|
}
|
||||||
@ -601,9 +604,6 @@ bool StorageChunkMerger::MergeTask::mergeChunks(const Storages & chunks)
|
|||||||
/// что-нибудь может сломаться.
|
/// что-нибудь может сломаться.
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Сейчас на new_storage ссылаются таблицы типа ChunkRef. Удалим лишнюю ссылку, которая была при создании.
|
|
||||||
new_storage.removeReference();
|
|
||||||
|
|
||||||
LOG_TRACE(log, "Merged chunks.");
|
LOG_TRACE(log, "Merged chunks.");
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
@ -613,6 +613,7 @@ bool StorageChunkMerger::MergeTask::mergeChunks(const Storages & chunks)
|
|||||||
Poco::ScopedLock<Poco::Mutex> lock(context.getMutex());
|
Poco::ScopedLock<Poco::Mutex> lock(context.getMutex());
|
||||||
|
|
||||||
currently_written_groups.erase(new_table_full_name);
|
currently_written_groups.erase(new_table_full_name);
|
||||||
|
executeQuery("DROP TABLE IF EXISTS " + new_table_full_name, context, true);
|
||||||
|
|
||||||
throw;
|
throw;
|
||||||
}
|
}
|
||||||
|
@ -161,9 +161,6 @@ StorageChunks::StorageChunks(
|
|||||||
context(context_),
|
context(context_),
|
||||||
log(&Logger::get("StorageChunks"))
|
log(&Logger::get("StorageChunks"))
|
||||||
{
|
{
|
||||||
if (!attach)
|
|
||||||
reference_counter.add(1, true);
|
|
||||||
|
|
||||||
_table_column_name = "_table" + VirtualColumnUtils::chooseSuffix(getColumnsList(), "_table");
|
_table_column_name = "_table" + VirtualColumnUtils::chooseSuffix(getColumnsList(), "_table");
|
||||||
|
|
||||||
try
|
try
|
||||||
|
@ -154,7 +154,7 @@ BlockInputStreams StorageDistributed::read(
|
|||||||
|
|
||||||
size_t result_size = (cluster.pools.size() * settings.max_parallel_replicas) + cluster.getLocalNodesNum();
|
size_t result_size = (cluster.pools.size() * settings.max_parallel_replicas) + cluster.getLocalNodesNum();
|
||||||
|
|
||||||
processed_stage = result_size == 1
|
processed_stage = result_size == 1 || settings.distributed_group_by_no_merge
|
||||||
? QueryProcessingStage::Complete
|
? QueryProcessingStage::Complete
|
||||||
: QueryProcessingStage::WithMergeableState;
|
: QueryProcessingStage::WithMergeableState;
|
||||||
|
|
||||||
|
@ -233,7 +233,15 @@ Block LogBlockInputStream::readImpl()
|
|||||||
else
|
else
|
||||||
column.column = column.type->createColumn();
|
column.column = column.type->createColumn();
|
||||||
|
|
||||||
readData(*it, *column.type, *column.column, max_rows_to_read, 0, read_offsets);
|
try
|
||||||
|
{
|
||||||
|
readData(*it, *column.type, *column.column, max_rows_to_read, 0, read_offsets);
|
||||||
|
}
|
||||||
|
catch (Exception & e)
|
||||||
|
{
|
||||||
|
e.addMessage("while reading column " + *it + " at " + storage.path + escapeForFileName(storage.name));
|
||||||
|
throw;
|
||||||
|
}
|
||||||
|
|
||||||
if (column.column->size())
|
if (column.column->size())
|
||||||
res.insert(column);
|
res.insert(column);
|
||||||
|
@ -198,7 +198,15 @@ Block TinyLogBlockInputStream::readImpl()
|
|||||||
else
|
else
|
||||||
column.column = column.type->createColumn();
|
column.column = column.type->createColumn();
|
||||||
|
|
||||||
readData(*it, *column.type, *column.column, block_size, 0, read_offsets);
|
try
|
||||||
|
{
|
||||||
|
readData(*it, *column.type, *column.column, block_size, 0, read_offsets);
|
||||||
|
}
|
||||||
|
catch (Exception & e)
|
||||||
|
{
|
||||||
|
e.addMessage("while reading column " + *it + " at " + storage.full_path());
|
||||||
|
throw;
|
||||||
|
}
|
||||||
|
|
||||||
if (column.column->size())
|
if (column.column->size())
|
||||||
res.insert(column);
|
res.insert(column);
|
||||||
|
18
dbms/tests/queries/0_stateless/00098_k_union_all.reference
Normal file
18
dbms/tests/queries/0_stateless/00098_k_union_all.reference
Normal file
@ -0,0 +1,18 @@
|
|||||||
|
[1m1[0m
|
||||||
|
|
||||||
|
1
|
||||||
|
[1m1[0m
|
||||||
|
|
||||||
|
1
|
||||||
|
[1m1[0m
|
||||||
|
|
||||||
|
1
|
||||||
|
[1m1[0m
|
||||||
|
|
||||||
|
1
|
||||||
|
[1m1[0m
|
||||||
|
|
||||||
|
1
|
||||||
|
[1m1[0m
|
||||||
|
|
||||||
|
1
|
3
dbms/tests/queries/0_stateless/00098_k_union_all.sql
Normal file
3
dbms/tests/queries/0_stateless/00098_k_union_all.sql
Normal file
@ -0,0 +1,3 @@
|
|||||||
|
SELECT 1 FORMAT PrettySpace;
|
||||||
|
SELECT 1 UNION ALL SELECT 1 FORMAT PrettySpace;
|
||||||
|
SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1 FORMAT PrettySpace;
|
@ -0,0 +1,12 @@
|
|||||||
|
DROP TABLE IF EXISTS test.t;
|
||||||
|
DROP TABLE IF EXISTS test.mv;
|
||||||
|
DROP TABLE IF EXISTS test.`.inner.mv`;
|
||||||
|
|
||||||
|
CREATE TABLE test.t (x UInt8) ENGINE = Null;
|
||||||
|
CREATE MATERIALIZED VIEW test.mv AS SELECT * FROM test.t;
|
||||||
|
|
||||||
|
DETACH TABLE test.mv;
|
||||||
|
ATTACH MATERIALIZED VIEW test.mv AS SELECT * FROM test.t;
|
||||||
|
|
||||||
|
DROP TABLE test.t;
|
||||||
|
DROP TABLE test.mv;
|
@ -0,0 +1,14 @@
|
|||||||
|
inf
|
||||||
|
0
|
||||||
|
inf
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
inf
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
inf
|
||||||
|
0
|
@ -0,0 +1,143 @@
|
|||||||
|
DROP TABLE IF EXISTS series;
|
||||||
|
|
||||||
|
CREATE TABLE series(i UInt32, x_value Float64, y_value Float64) ENGINE = Memory;
|
||||||
|
|
||||||
|
/* Тестовые данные */
|
||||||
|
|
||||||
|
INSERT INTO series(i, x_value, y_value) VALUES (1, 5.6,-4.4),(2, -9.6,3),(3, -1.3,-4),(4, 5.3,9.7),(5, 4.4,0.037),(6, -8.6,-7.8),(7, 5.1,9.3),(8, 7.9,-3.6),(9, -8.2,0.62),(10, -3,7.3);
|
||||||
|
|
||||||
|
/* varSamp */
|
||||||
|
|
||||||
|
SELECT varSamp(x_value) FROM (SELECT x_value FROM series LIMIT 0);
|
||||||
|
SELECT varSamp(x_value) FROM (SELECT x_value FROM series LIMIT 1);
|
||||||
|
|
||||||
|
SELECT round(abs(res1 - res2), 6) FROM
|
||||||
|
(
|
||||||
|
SELECT
|
||||||
|
varSamp(x_value) AS res1,
|
||||||
|
(sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / (count() - 1) AS res2
|
||||||
|
FROM series
|
||||||
|
);
|
||||||
|
|
||||||
|
/* stddevSamp */
|
||||||
|
|
||||||
|
SELECT stddevSamp(x_value) FROM (SELECT x_value FROM series LIMIT 0);
|
||||||
|
SELECT stddevSamp(x_value) FROM (SELECT x_value FROM series LIMIT 1);
|
||||||
|
|
||||||
|
SELECT round(abs(res1 - res2), 6) FROM
|
||||||
|
(
|
||||||
|
SELECT
|
||||||
|
stddevSamp(x_value) AS res1,
|
||||||
|
sqrt((sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / (count() - 1)) AS res2
|
||||||
|
FROM series
|
||||||
|
);
|
||||||
|
|
||||||
|
/* varPop */
|
||||||
|
|
||||||
|
SELECT varPop(x_value) FROM (SELECT x_value FROM series LIMIT 0);
|
||||||
|
SELECT varPop(x_value) FROM (SELECT x_value FROM series LIMIT 1);
|
||||||
|
|
||||||
|
SELECT round(abs(res1 - res2), 6) FROM
|
||||||
|
(
|
||||||
|
SELECT
|
||||||
|
varPop(x_value) AS res1,
|
||||||
|
(sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / count() AS res2
|
||||||
|
FROM series
|
||||||
|
);
|
||||||
|
|
||||||
|
/* stddevPop */
|
||||||
|
|
||||||
|
SELECT stddevPop(x_value) FROM (SELECT x_value FROM series LIMIT 0);
|
||||||
|
SELECT stddevPop(x_value) FROM (SELECT x_value FROM series LIMIT 1);
|
||||||
|
|
||||||
|
SELECT round(abs(res1 - res2), 6) FROM
|
||||||
|
(
|
||||||
|
SELECT
|
||||||
|
stddevPop(x_value) AS res1,
|
||||||
|
sqrt((sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / count()) AS res2
|
||||||
|
FROM series
|
||||||
|
);
|
||||||
|
|
||||||
|
/* covarSamp */
|
||||||
|
|
||||||
|
SELECT covarSamp(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 0);
|
||||||
|
SELECT covarSamp(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 1);
|
||||||
|
|
||||||
|
SELECT round(abs(COVAR1 - COVAR2), 6)
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT
|
||||||
|
arrayJoin([1]) AS ID2,
|
||||||
|
covarSamp(x_value, y_value) AS COVAR1
|
||||||
|
FROM series
|
||||||
|
) ANY INNER JOIN
|
||||||
|
(
|
||||||
|
SELECT
|
||||||
|
arrayJoin([1]) AS ID2,
|
||||||
|
sum(VAL) / (count() - 1) AS COVAR2
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT (X - AVG_X) * (Y - AVG_Y) AS VAL
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT
|
||||||
|
toUInt32(arrayJoin([1, 2, 3, 4, 5, 6, 7, 8, 9, 10])) AS ID,
|
||||||
|
avg(x_value) AS AVG_X,
|
||||||
|
avg(y_value) AS AVG_Y
|
||||||
|
FROM series
|
||||||
|
) ANY INNER JOIN
|
||||||
|
(
|
||||||
|
SELECT
|
||||||
|
i AS ID,
|
||||||
|
x_value AS X,
|
||||||
|
y_value AS Y
|
||||||
|
FROM series
|
||||||
|
) USING ID
|
||||||
|
)
|
||||||
|
) USING ID2;
|
||||||
|
|
||||||
|
/* covarPop */
|
||||||
|
|
||||||
|
SELECT covarPop(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 0);
|
||||||
|
SELECT covarPop(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 1);
|
||||||
|
|
||||||
|
SELECT round(abs(COVAR1 - COVAR2), 6)
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT
|
||||||
|
arrayJoin([1]) AS ID2,
|
||||||
|
covarPop(x_value, y_value) AS COVAR1
|
||||||
|
FROM series
|
||||||
|
) ANY INNER JOIN
|
||||||
|
(
|
||||||
|
SELECT
|
||||||
|
arrayJoin([1]) AS ID2,
|
||||||
|
sum(VAL) / count() AS COVAR2
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT (X - AVG_X) * (Y - AVG_Y) AS VAL
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT
|
||||||
|
toUInt32(arrayJoin([1, 2, 3, 4, 5, 6, 7, 8, 9, 10])) AS ID,
|
||||||
|
avg(x_value) AS AVG_X,
|
||||||
|
avg(y_value) AS AVG_Y
|
||||||
|
FROM series
|
||||||
|
) ANY INNER JOIN
|
||||||
|
(
|
||||||
|
SELECT
|
||||||
|
i AS ID,
|
||||||
|
x_value AS X,
|
||||||
|
y_value AS Y
|
||||||
|
FROM series
|
||||||
|
) USING ID
|
||||||
|
)
|
||||||
|
) USING ID2;
|
||||||
|
|
||||||
|
/* corr */
|
||||||
|
|
||||||
|
SELECT corr(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 0);
|
||||||
|
SELECT corr(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 1);
|
||||||
|
|
||||||
|
SELECT round(abs(corr(x_value, y_value) - covarPop(x_value, y_value) / (stddevPop(x_value) * stddevPop(y_value))), 6) FROM series;
|
||||||
|
|
@ -0,0 +1,914 @@
|
|||||||
|
---map--
|
||||||
|
[]
|
||||||
|
[123,123,123]
|
||||||
|
[]
|
||||||
|
[123]
|
||||||
|
[123,123]
|
||||||
|
[123,123,123]
|
||||||
|
[123,123,123,123]
|
||||||
|
[123,123,123,123,123]
|
||||||
|
[123,123,123,123,123,123]
|
||||||
|
[123,123,123,123,123,123,123]
|
||||||
|
[123,123,123,123,123,123,123,123]
|
||||||
|
[123,123,123,123,123,123,123,123,123]
|
||||||
|
---filter--
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[1,2,3]
|
||||||
|
[]
|
||||||
|
[0]
|
||||||
|
[0,1]
|
||||||
|
[0,1,2]
|
||||||
|
[0,1,2,3]
|
||||||
|
[0,1,2,3,4]
|
||||||
|
[0,1,2,3,4,5]
|
||||||
|
[0,1,2,3,4,5,6]
|
||||||
|
[0,1,2,3,4,5,6,7]
|
||||||
|
[0,1,2,3,4,5,6,7,8]
|
||||||
|
---count---
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
3
|
||||||
|
0
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
4
|
||||||
|
5
|
||||||
|
6
|
||||||
|
7
|
||||||
|
8
|
||||||
|
9
|
||||||
|
---sum---
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
30
|
||||||
|
0
|
||||||
|
10
|
||||||
|
20
|
||||||
|
30
|
||||||
|
40
|
||||||
|
50
|
||||||
|
60
|
||||||
|
70
|
||||||
|
80
|
||||||
|
90
|
||||||
|
---all---
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
---exists---
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
---first---
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
---first index---
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
---map--
|
||||||
|
[]
|
||||||
|
[123,123,123]
|
||||||
|
[]
|
||||||
|
[123]
|
||||||
|
[123,123]
|
||||||
|
[123,123,123]
|
||||||
|
[123,123,123,123]
|
||||||
|
[123,123,123,123,123]
|
||||||
|
[123,123,123,123,123,123]
|
||||||
|
[123,123,123,123,123,123,123]
|
||||||
|
[123,123,123,123,123,123,123,123]
|
||||||
|
[123,123,123,123,123,123,123,123,123]
|
||||||
|
---filter--
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[1,2,3]
|
||||||
|
[]
|
||||||
|
[0]
|
||||||
|
[0,1]
|
||||||
|
[0,1,2]
|
||||||
|
[0,1,2,3]
|
||||||
|
[0,1,2,3,4]
|
||||||
|
[0,1,2,3,4,5]
|
||||||
|
[0,1,2,3,4,5,6]
|
||||||
|
[0,1,2,3,4,5,6,7]
|
||||||
|
[0,1,2,3,4,5,6,7,8]
|
||||||
|
---count---
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
3
|
||||||
|
0
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
4
|
||||||
|
5
|
||||||
|
6
|
||||||
|
7
|
||||||
|
8
|
||||||
|
9
|
||||||
|
---sum---
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
30
|
||||||
|
0
|
||||||
|
10
|
||||||
|
20
|
||||||
|
30
|
||||||
|
40
|
||||||
|
50
|
||||||
|
60
|
||||||
|
70
|
||||||
|
80
|
||||||
|
90
|
||||||
|
---all---
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
---exists---
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
---first---
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
---first index---
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
---map--
|
||||||
|
[]
|
||||||
|
[123,123,123]
|
||||||
|
[]
|
||||||
|
[123]
|
||||||
|
[123,123]
|
||||||
|
[123,123,123]
|
||||||
|
[123,123,123,123]
|
||||||
|
[123,123,123,123,123]
|
||||||
|
[123,123,123,123,123,123]
|
||||||
|
[123,123,123,123,123,123,123]
|
||||||
|
[123,123,123,123,123,123,123,123]
|
||||||
|
[123,123,123,123,123,123,123,123,123]
|
||||||
|
---filter--
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
['1','2','3']
|
||||||
|
[]
|
||||||
|
['0']
|
||||||
|
['0','1']
|
||||||
|
['0','1','2']
|
||||||
|
['0','1','2','3']
|
||||||
|
['0','1','2','3','4']
|
||||||
|
['0','1','2','3','4','5']
|
||||||
|
['0','1','2','3','4','5','6']
|
||||||
|
['0','1','2','3','4','5','6','7']
|
||||||
|
['0','1','2','3','4','5','6','7','8']
|
||||||
|
---count---
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
3
|
||||||
|
0
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
4
|
||||||
|
5
|
||||||
|
6
|
||||||
|
7
|
||||||
|
8
|
||||||
|
9
|
||||||
|
---sum---
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
30
|
||||||
|
0
|
||||||
|
10
|
||||||
|
20
|
||||||
|
30
|
||||||
|
40
|
||||||
|
50
|
||||||
|
60
|
||||||
|
70
|
||||||
|
80
|
||||||
|
90
|
||||||
|
---all---
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
---exists---
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
---first---
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
1
|
||||||
|
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
---first index---
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
---map--
|
||||||
|
[]
|
||||||
|
[123,123,123]
|
||||||
|
[]
|
||||||
|
[123]
|
||||||
|
[123,123]
|
||||||
|
[123,123,123]
|
||||||
|
[123,123,123,123]
|
||||||
|
[123,123,123,123,123]
|
||||||
|
[123,123,123,123,123,123]
|
||||||
|
[123,123,123,123,123,123,123]
|
||||||
|
[123,123,123,123,123,123,123,123]
|
||||||
|
[123,123,123,123,123,123,123,123,123]
|
||||||
|
---filter--
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
[]
|
||||||
|
['1','2','3']
|
||||||
|
[]
|
||||||
|
['0']
|
||||||
|
['0','1']
|
||||||
|
['0','1','2']
|
||||||
|
['0','1','2','3']
|
||||||
|
['0','1','2','3','4']
|
||||||
|
['0','1','2','3','4','5']
|
||||||
|
['0','1','2','3','4','5','6']
|
||||||
|
['0','1','2','3','4','5','6','7']
|
||||||
|
['0','1','2','3','4','5','6','7','8']
|
||||||
|
---count---
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
3
|
||||||
|
0
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
4
|
||||||
|
5
|
||||||
|
6
|
||||||
|
7
|
||||||
|
8
|
||||||
|
9
|
||||||
|
---sum---
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
30
|
||||||
|
0
|
||||||
|
10
|
||||||
|
20
|
||||||
|
30
|
||||||
|
40
|
||||||
|
50
|
||||||
|
60
|
||||||
|
70
|
||||||
|
80
|
||||||
|
90
|
||||||
|
---all---
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
---exists---
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
---first---
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
1
|
||||||
|
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
---first index---
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
--- ---
|
||||||
|
[]
|
||||||
|
[1]
|
||||||
|
[0,0]
|
||||||
|
[1,1,1]
|
||||||
|
[0,0,0,0]
|
||||||
|
[1,1,1,1,1]
|
||||||
|
[0,0,0,0,0,0]
|
||||||
|
[1,1,1,1,1,1,1]
|
||||||
|
[0,0,0,0,0,0,0,0]
|
||||||
|
[1,1,1,1,1,1,1,1,1]
|
||||||
|
[]
|
||||||
|
[0]
|
||||||
|
[]
|
||||||
|
[0,1,2]
|
||||||
|
[]
|
||||||
|
[0,1,2,3,4]
|
||||||
|
[]
|
||||||
|
[0,1,2,3,4,5,6]
|
||||||
|
[]
|
||||||
|
[0,1,2,3,4,5,6,7,8]
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
3
|
||||||
|
0
|
||||||
|
5
|
||||||
|
0
|
||||||
|
7
|
||||||
|
0
|
||||||
|
9
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
3
|
||||||
|
0
|
||||||
|
5
|
||||||
|
0
|
||||||
|
7
|
||||||
|
0
|
||||||
|
9
|
||||||
|
1
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
--- ---
|
||||||
|
[]
|
||||||
|
[1]
|
||||||
|
[0,0]
|
||||||
|
[1,1,1]
|
||||||
|
[0,0,0,0]
|
||||||
|
[1,1,1,1,1]
|
||||||
|
[0,0,0,0,0,0]
|
||||||
|
[1,1,1,1,1,1,1]
|
||||||
|
[0,0,0,0,0,0,0,0]
|
||||||
|
[1,1,1,1,1,1,1,1,1]
|
||||||
|
[]
|
||||||
|
['0']
|
||||||
|
[]
|
||||||
|
['0','1','2']
|
||||||
|
[]
|
||||||
|
['0','1','2','3','4']
|
||||||
|
[]
|
||||||
|
['0','1','2','3','4','5','6']
|
||||||
|
[]
|
||||||
|
['0','1','2','3','4','5','6','7','8']
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
3
|
||||||
|
0
|
||||||
|
5
|
||||||
|
0
|
||||||
|
7
|
||||||
|
0
|
||||||
|
9
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
3
|
||||||
|
0
|
||||||
|
5
|
||||||
|
0
|
||||||
|
7
|
||||||
|
0
|
||||||
|
9
|
||||||
|
1
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
|
||||||
|
0
|
||||||
|
|
||||||
|
0
|
||||||
|
|
||||||
|
0
|
||||||
|
|
||||||
|
0
|
||||||
|
|
||||||
|
0
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
@ -0,0 +1,235 @@
|
|||||||
|
SELECT '---map--';
|
||||||
|
SELECT arrayMap(x -> 123, emptyArrayUInt8());
|
||||||
|
SELECT arrayMap(x -> 123, [1, 2, 3]);
|
||||||
|
SELECT arrayMap(x -> 123, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---filter--';
|
||||||
|
SELECT arrayFilter(x -> 0, emptyArrayUInt8());
|
||||||
|
SELECT arrayFilter(x -> 0, [1, 2, 3]);
|
||||||
|
SELECT arrayFilter(x -> 0, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayFilter(x -> 1, emptyArrayUInt8());
|
||||||
|
SELECT arrayFilter(x -> 1, [1, 2, 3]);
|
||||||
|
SELECT arrayFilter(x -> 1, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---count---';
|
||||||
|
SELECT arrayCount(x -> 0, emptyArrayUInt8());
|
||||||
|
SELECT arrayCount(x -> 0, [1, 2, 3]);
|
||||||
|
SELECT arrayCount(x -> 0, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayCount(x -> 1, emptyArrayUInt8());
|
||||||
|
SELECT arrayCount(x -> 1, [1, 2, 3]);
|
||||||
|
SELECT arrayCount(x -> 1, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---sum---';
|
||||||
|
SELECT arraySum(x -> 0, emptyArrayUInt8());
|
||||||
|
SELECT arraySum(x -> 0, [1, 2, 3]);
|
||||||
|
SELECT arraySum(x -> 0, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arraySum(x -> 10, emptyArrayUInt8());
|
||||||
|
SELECT arraySum(x -> 10, [1, 2, 3]);
|
||||||
|
SELECT arraySum(x -> 10, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---all---';
|
||||||
|
SELECT arrayAll(x -> 0, emptyArrayUInt8());
|
||||||
|
SELECT arrayAll(x -> 0, [1, 2, 3]);
|
||||||
|
SELECT arrayAll(x -> 0, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayAll(x -> 1, emptyArrayUInt8());
|
||||||
|
SELECT arrayAll(x -> 1, [1, 2, 3]);
|
||||||
|
SELECT arrayAll(x -> 1, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---exists---';
|
||||||
|
SELECT arrayExists(x -> 0, emptyArrayUInt8());
|
||||||
|
SELECT arrayExists(x -> 0, [1, 2, 3]);
|
||||||
|
SELECT arrayExists(x -> 0, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayExists(x -> 1, emptyArrayUInt8());
|
||||||
|
SELECT arrayExists(x -> 1, [1, 2, 3]);
|
||||||
|
SELECT arrayExists(x -> 1, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---first---';
|
||||||
|
SELECT arrayFirst(x -> 0, emptyArrayUInt8());
|
||||||
|
SELECT arrayFirst(x -> 0, [1, 2, 3]);
|
||||||
|
SELECT arrayFirst(x -> 0, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayFirst(x -> 1, emptyArrayUInt8());
|
||||||
|
SELECT arrayFirst(x -> 1, [1, 2, 3]);
|
||||||
|
SELECT arrayFirst(x -> 1, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---first index---';
|
||||||
|
SELECT arrayFirstIndex(x -> 0, emptyArrayUInt8());
|
||||||
|
SELECT arrayFirstIndex(x -> 0, [1, 2, 3]);
|
||||||
|
SELECT arrayFirstIndex(x -> 0, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayFirstIndex(x -> 1, emptyArrayUInt8());
|
||||||
|
SELECT arrayFirstIndex(x -> 1, [1, 2, 3]);
|
||||||
|
SELECT arrayFirstIndex(x -> 1, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
|
||||||
|
SELECT '---map--';
|
||||||
|
SELECT arrayMap(x -> materialize(123), emptyArrayUInt8());
|
||||||
|
SELECT arrayMap(x -> materialize(123), [1, 2, 3]);
|
||||||
|
SELECT arrayMap(x -> materialize(123), range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---filter--';
|
||||||
|
SELECT arrayFilter(x -> materialize(0), emptyArrayUInt8());
|
||||||
|
SELECT arrayFilter(x -> materialize(0), [1, 2, 3]);
|
||||||
|
SELECT arrayFilter(x -> materialize(0), range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayFilter(x -> materialize(1), emptyArrayUInt8());
|
||||||
|
SELECT arrayFilter(x -> materialize(1), [1, 2, 3]);
|
||||||
|
SELECT arrayFilter(x -> materialize(1), range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---count---';
|
||||||
|
SELECT arrayCount(x -> materialize(0), emptyArrayUInt8());
|
||||||
|
SELECT arrayCount(x -> materialize(0), [1, 2, 3]);
|
||||||
|
SELECT arrayCount(x -> materialize(0), range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayCount(x -> materialize(1), emptyArrayUInt8());
|
||||||
|
SELECT arrayCount(x -> materialize(1), [1, 2, 3]);
|
||||||
|
SELECT arrayCount(x -> materialize(1), range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---sum---';
|
||||||
|
SELECT arraySum(x -> materialize(0), emptyArrayUInt8());
|
||||||
|
SELECT arraySum(x -> materialize(0), [1, 2, 3]);
|
||||||
|
SELECT arraySum(x -> materialize(0), range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arraySum(x -> materialize(10), emptyArrayUInt8());
|
||||||
|
SELECT arraySum(x -> materialize(10), [1, 2, 3]);
|
||||||
|
SELECT arraySum(x -> materialize(10), range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---all---';
|
||||||
|
SELECT arrayAll(x -> materialize(0), emptyArrayUInt8());
|
||||||
|
SELECT arrayAll(x -> materialize(0), [1, 2, 3]);
|
||||||
|
SELECT arrayAll(x -> materialize(0), range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayAll(x -> materialize(1), emptyArrayUInt8());
|
||||||
|
SELECT arrayAll(x -> materialize(1), [1, 2, 3]);
|
||||||
|
SELECT arrayAll(x -> materialize(1), range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---exists---';
|
||||||
|
SELECT arrayExists(x -> materialize(0), emptyArrayUInt8());
|
||||||
|
SELECT arrayExists(x -> materialize(0), [1, 2, 3]);
|
||||||
|
SELECT arrayExists(x -> materialize(0), range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayExists(x -> materialize(1), emptyArrayUInt8());
|
||||||
|
SELECT arrayExists(x -> materialize(1), [1, 2, 3]);
|
||||||
|
SELECT arrayExists(x -> materialize(1), range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---first---';
|
||||||
|
SELECT arrayFirst(x -> materialize(0), emptyArrayUInt8());
|
||||||
|
SELECT arrayFirst(x -> materialize(0), [1, 2, 3]);
|
||||||
|
SELECT arrayFirst(x -> materialize(0), range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayFirst(x -> materialize(1), emptyArrayUInt8());
|
||||||
|
SELECT arrayFirst(x -> materialize(1), [1, 2, 3]);
|
||||||
|
SELECT arrayFirst(x -> materialize(1), range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---first index---';
|
||||||
|
SELECT arrayFirstIndex(x -> materialize(0), emptyArrayUInt8());
|
||||||
|
SELECT arrayFirstIndex(x -> materialize(0), [1, 2, 3]);
|
||||||
|
SELECT arrayFirstIndex(x -> materialize(0), range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayFirstIndex(x -> materialize(1), emptyArrayUInt8());
|
||||||
|
SELECT arrayFirstIndex(x -> materialize(1), [1, 2, 3]);
|
||||||
|
SELECT arrayFirstIndex(x -> materialize(1), range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
|
||||||
|
SELECT '---map--';
|
||||||
|
SELECT arrayMap(x -> 123, emptyArrayString());
|
||||||
|
SELECT arrayMap(x -> 123, arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayMap(x -> 123, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---filter--';
|
||||||
|
SELECT arrayFilter(x -> 0, emptyArrayString());
|
||||||
|
SELECT arrayFilter(x -> 0, arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayFilter(x -> 0, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayFilter(x -> 1, emptyArrayString());
|
||||||
|
SELECT arrayFilter(x -> 1, arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayFilter(x -> 1, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---count---';
|
||||||
|
SELECT arrayCount(x -> 0, emptyArrayString());
|
||||||
|
SELECT arrayCount(x -> 0, arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayCount(x -> 0, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayCount(x -> 1, emptyArrayString());
|
||||||
|
SELECT arrayCount(x -> 1, arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayCount(x -> 1, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---sum---';
|
||||||
|
SELECT arraySum(x -> 0, emptyArrayString());
|
||||||
|
SELECT arraySum(x -> 0, arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arraySum(x -> 0, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arraySum(x -> 10, emptyArrayString());
|
||||||
|
SELECT arraySum(x -> 10, arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arraySum(x -> 10, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---all---';
|
||||||
|
SELECT arrayAll(x -> 0, emptyArrayString());
|
||||||
|
SELECT arrayAll(x -> 0, arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayAll(x -> 0, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayAll(x -> 1, emptyArrayString());
|
||||||
|
SELECT arrayAll(x -> 1, arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayAll(x -> 1, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---exists---';
|
||||||
|
SELECT arrayExists(x -> 0, emptyArrayString());
|
||||||
|
SELECT arrayExists(x -> 0, arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayExists(x -> 0, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayExists(x -> 1, emptyArrayString());
|
||||||
|
SELECT arrayExists(x -> 1, arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayExists(x -> 1, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---first---';
|
||||||
|
SELECT arrayFirst(x -> 0, emptyArrayString());
|
||||||
|
SELECT arrayFirst(x -> 0, arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayFirst(x -> 0, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayFirst(x -> 1, emptyArrayString());
|
||||||
|
SELECT arrayFirst(x -> 1, arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayFirst(x -> 1, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---first index---';
|
||||||
|
SELECT arrayFirstIndex(x -> 0, emptyArrayString());
|
||||||
|
SELECT arrayFirstIndex(x -> 0, arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayFirstIndex(x -> 0, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayFirstIndex(x -> 1, emptyArrayString());
|
||||||
|
SELECT arrayFirstIndex(x -> 1, arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayFirstIndex(x -> 1, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
|
||||||
|
SELECT '---map--';
|
||||||
|
SELECT arrayMap(x -> materialize(123), emptyArrayString());
|
||||||
|
SELECT arrayMap(x -> materialize(123), arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayMap(x -> materialize(123), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---filter--';
|
||||||
|
SELECT arrayFilter(x -> materialize(0), emptyArrayString());
|
||||||
|
SELECT arrayFilter(x -> materialize(0), arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayFilter(x -> materialize(0), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayFilter(x -> materialize(1), emptyArrayString());
|
||||||
|
SELECT arrayFilter(x -> materialize(1), arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayFilter(x -> materialize(1), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---count---';
|
||||||
|
SELECT arrayCount(x -> materialize(0), emptyArrayString());
|
||||||
|
SELECT arrayCount(x -> materialize(0), arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayCount(x -> materialize(0), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayCount(x -> materialize(1), emptyArrayString());
|
||||||
|
SELECT arrayCount(x -> materialize(1), arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayCount(x -> materialize(1), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---sum---';
|
||||||
|
SELECT arraySum(x -> materialize(0), emptyArrayString());
|
||||||
|
SELECT arraySum(x -> materialize(0), arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arraySum(x -> materialize(0), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arraySum(x -> materialize(10), emptyArrayString());
|
||||||
|
SELECT arraySum(x -> materialize(10), arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arraySum(x -> materialize(10), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---all---';
|
||||||
|
SELECT arrayAll(x -> materialize(0), emptyArrayString());
|
||||||
|
SELECT arrayAll(x -> materialize(0), arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayAll(x -> materialize(0), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayAll(x -> materialize(1), emptyArrayString());
|
||||||
|
SELECT arrayAll(x -> materialize(1), arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayAll(x -> materialize(1), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---exists---';
|
||||||
|
SELECT arrayExists(x -> materialize(0), emptyArrayString());
|
||||||
|
SELECT arrayExists(x -> materialize(0), arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayExists(x -> materialize(0), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayExists(x -> materialize(1), emptyArrayString());
|
||||||
|
SELECT arrayExists(x -> materialize(1), arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayExists(x -> materialize(1), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---first---';
|
||||||
|
SELECT arrayFirst(x -> materialize(0), emptyArrayString());
|
||||||
|
SELECT arrayFirst(x -> materialize(0), arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayFirst(x -> materialize(0), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayFirst(x -> materialize(1), emptyArrayString());
|
||||||
|
SELECT arrayFirst(x -> materialize(1), arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayFirst(x -> materialize(1), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '---first index---';
|
||||||
|
SELECT arrayFirstIndex(x -> materialize(0), emptyArrayString());
|
||||||
|
SELECT arrayFirstIndex(x -> materialize(0), arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayFirstIndex(x -> materialize(0), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayFirstIndex(x -> materialize(1), emptyArrayString());
|
||||||
|
SELECT arrayFirstIndex(x -> materialize(1), arrayMap(x -> toString(x), [1, 2, 3]));
|
||||||
|
SELECT arrayFirstIndex(x -> materialize(1), arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
|
||||||
|
|
||||||
|
SELECT '--- ---';
|
||||||
|
SELECT arrayMap(x -> number % 2, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayFilter(x -> number % 2, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayCount(x -> number % 2, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arraySum(x -> number % 2, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayAll(x -> number % 2, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayExists(x -> number % 2, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayFirst(x -> number % 2, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayFirstIndex(x -> number % 2, range(number)) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT '--- ---';
|
||||||
|
SELECT arrayMap(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayFilter(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayCount(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arraySum(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayAll(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayExists(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayFirst(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
||||||
|
SELECT arrayFirstIndex(x -> number % 2, arrayMap(x -> toString(x), range(number))) FROM system.numbers LIMIT 10;
|
@ -0,0 +1 @@
|
|||||||
|
2
|
@ -0,0 +1 @@
|
|||||||
|
SELECT count() FROM remote('{127,1}.0.0.{1,2}', system.one) SETTINGS skip_unavailable_shards = 1;
|
@ -0,0 +1,2 @@
|
|||||||
|
1 1
|
||||||
|
1 1
|
@ -0,0 +1 @@
|
|||||||
|
SELECT count(), uniq(dummy) FROM remote('127.0.0.{1,2}', system.one) SETTINGS distributed_group_by_no_merge = 1;
|
@ -0,0 +1,23 @@
|
|||||||
|
[1,2]
|
||||||
|
[1,2]
|
||||||
|
[-1,2]
|
||||||
|
[-1,1000]
|
||||||
|
[-1,1000000]
|
||||||
|
[-1000,2]
|
||||||
|
[-1000000,2]
|
||||||
|
['Hello','world']
|
||||||
|
[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121,122,123,124,125,126,127,128,129,130,131,132,133,134,135,136,137,138,139,140,141,142,143,144,145,146,147,148,149,150,151,152,153,154,155,156,157,158,159,160,161,162,163,164,165,166,167,168,169,170,171,172,173,174,175,176,177,178,179,180,181,182,183,184,185,186,187,188,189,190,191,192,193,194,195,196,197,198,199,200,201,202,203,204,205,206,207,208,209,210,211,212,213,214,215,216,217,218,219,220,221,222,223,224,225,226,227,228,229,230,231,232,233,234,235,236,237,238,239,240,241,242,243,244,245,246,247,248,249,250,251,252,253,254,255,256]
|
||||||
|
[1,2]
|
||||||
|
[1,2]
|
||||||
|
[-1,2]
|
||||||
|
[-1,1000]
|
||||||
|
[-1,1000000]
|
||||||
|
[-1000,2]
|
||||||
|
[-1000000,2]
|
||||||
|
['Hello','world']
|
||||||
|
[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121,122,123,124,125,126,127,128,129,130,131,132,133,134,135,136,137,138,139,140,141,142,143,144,145,146,147,148,149,150,151,152,153,154,155,156,157,158,159,160,161,162,163,164,165,166,167,168,169,170,171,172,173,174,175,176,177,178,179,180,181,182,183,184,185,186,187,188,189,190,191,192,193,194,195,196,197,198,199,200,201,202,203,204,205,206,207,208,209,210,211,212,213,214,215,216,217,218,219,220,221,222,223,224,225,226,227,228,229,230,231,232,233,234,235,236,237,238,239,240,241,242,243,244,245,246,247,248,249,250,251,252,253,254,255,256]
|
||||||
|
[0]
|
||||||
|
[0]
|
||||||
|
[0]
|
||||||
|
[0]
|
||||||
|
[0]
|
30
dbms/tests/queries/0_stateless/00185_array_literals.sql
Normal file
30
dbms/tests/queries/0_stateless/00185_array_literals.sql
Normal file
@ -0,0 +1,30 @@
|
|||||||
|
SELECT [1, 2];
|
||||||
|
SELECT [1.0, 2];
|
||||||
|
SELECT [-1, 2];
|
||||||
|
SELECT [-1, 1000];
|
||||||
|
SELECT [-1, 1000000];
|
||||||
|
SELECT [-1000, 2];
|
||||||
|
SELECT [-1000000, 2];
|
||||||
|
SELECT ['Hello', 'world'];
|
||||||
|
|
||||||
|
SELECT [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121,122,123,124,125,126,127,128,129,130,131,132,133,134,135,136,137,138,139,140,141,142,143,144,145,146,147,148,149,150,151,152,153,154,155,156,157,158,159,160,161,162,163,164,165,166,167,168,169,170,171,172,173,174,175,176,177,178,179,180,181,182,183,184,185,186,187,188,189,190,191,192,193,194,195,196,197,198,199,200,201,202,203,204,205,206,207,208,209,210,211,212,213,214,215,216,217,218,219,220,221,222,223,224,225,226,227,228,229,230,231,232,233,234,235,236,237,238,239,240,241,242,243,244,245,246,247,248,249,250,251,252,253,254,255,256];
|
||||||
|
|
||||||
|
SELECT [1, 1 + 1];
|
||||||
|
SELECT [1.0, 1 + 1];
|
||||||
|
SELECT [-1, 1 + 1];
|
||||||
|
SELECT [-1, toUInt16(1000)];
|
||||||
|
SELECT [-1, toUInt32(1000000)];
|
||||||
|
SELECT [-1000, 1 + 1];
|
||||||
|
SELECT [-1000000, 1 + 1];
|
||||||
|
SELECT ['Hello', concat('wor', 'ld')];
|
||||||
|
|
||||||
|
SELECT [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121,122,123,124,125,126,127,128,129,130,131,132,133,134,135,136,137,138,139,140,141,142,143,144,145,146,147,148,149,150,151,152,153,154,155,156,157,158,159,160,161,162,163,164,165,166,167,168,169,170,171,172,173,174,175,176,177,178,179,180,181,182,183,184,185,186,187,188,189,190,191,192,193,194,195,196,197,198,199,200,201,202,203,204,205,206,207,208,209,210,211,212,213,214,215,216,217,218,219,220,221,222,223,224,225,226,227,228,229,230,231,232,233,234,235,236,237,238,239,240,241,242,243,244,245,246,247,248,249,250,251,252,253,254,255,256 + 0];
|
||||||
|
|
||||||
|
SELECT [0];
|
||||||
|
SELECT [0 ];
|
||||||
|
SELECT [ 0];
|
||||||
|
SELECT [ 0 ];
|
||||||
|
SELECT
|
||||||
|
[
|
||||||
|
0
|
||||||
|
];
|
@ -0,0 +1 @@
|
|||||||
|
10000000
|
3
dbms/tests/queries/0_stateless/00186_very_long_arrays.sh
Executable file
3
dbms/tests/queries/0_stateless/00186_very_long_arrays.sh
Executable file
@ -0,0 +1,3 @@
|
|||||||
|
#!/bin/bash
|
||||||
|
|
||||||
|
(echo 'SELECT number FROM system.numbers WHERE transform(number, ['; seq 1 100000 | tr '\n' ','; echo '0],['; seq 1 100000 | tr '\n' ','; echo '0]) = 10000000 LIMIT 1';) | clickhouse-client --max_query_size=100000000
|
@ -0,0 +1 @@
|
|||||||
|
1
|
@ -0,0 +1 @@
|
|||||||
|
SELECT materialize('prepre_f') LIKE '%pre_f%';
|
@ -0,0 +1 @@
|
|||||||
|
10 10 1
|
@ -0,0 +1 @@
|
|||||||
|
SELECT count(), sum(1), uniq(123) FROM (SELECT * FROM system.numbers LIMIT 10);
|
@ -47,7 +47,6 @@ public:
|
|||||||
</zookeeper>
|
</zookeeper>
|
||||||
*/
|
*/
|
||||||
ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std::string & config_name);
|
ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std::string & config_name);
|
||||||
ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std::string & config_name, int32_t session_timeout_ms);
|
|
||||||
|
|
||||||
~ZooKeeper();
|
~ZooKeeper();
|
||||||
|
|
||||||
@ -126,6 +125,9 @@ public:
|
|||||||
void set(const std::string & path, const std::string & data,
|
void set(const std::string & path, const std::string & data,
|
||||||
int32_t version = -1, Stat * stat = nullptr);
|
int32_t version = -1, Stat * stat = nullptr);
|
||||||
|
|
||||||
|
/** Создает ноду, если ее не существует. Иначе обновляет */
|
||||||
|
void createOrUpdate(const std::string & path, const std::string & data, int32_t mode);
|
||||||
|
|
||||||
/** Не бросает исключение при следующих ошибках:
|
/** Не бросает исключение при следующих ошибках:
|
||||||
* - Такой ноды нет.
|
* - Такой ноды нет.
|
||||||
* - У ноды другая версия.
|
* - У ноды другая версия.
|
||||||
|
@ -128,13 +128,6 @@ ZooKeeper::ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std
|
|||||||
init(args.hosts, args.session_timeout_ms);
|
init(args.hosts, args.session_timeout_ms);
|
||||||
}
|
}
|
||||||
|
|
||||||
ZooKeeper::ZooKeeper(const Poco::Util::AbstractConfiguration& config, const std::string& config_name, int32_t session_timeout_ms_)
|
|
||||||
{
|
|
||||||
ZooKeeperArgs args(config, config_name);
|
|
||||||
init(args.hosts, session_timeout_ms_);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
void * ZooKeeper::watchForEvent(EventPtr event)
|
void * ZooKeeper::watchForEvent(EventPtr event)
|
||||||
{
|
{
|
||||||
if (event)
|
if (event)
|
||||||
@ -390,7 +383,7 @@ bool ZooKeeper::tryGet(const std::string & path, std::string & res, Stat * stat_
|
|||||||
{
|
{
|
||||||
int32_t code = retry(std::bind(&ZooKeeper::getImpl, this, std::ref(path), std::ref(res), stat_, watch));
|
int32_t code = retry(std::bind(&ZooKeeper::getImpl, this, std::ref(path), std::ref(res), stat_, watch));
|
||||||
|
|
||||||
if (!( code == ZOK ||
|
if (!(code == ZOK ||
|
||||||
code == ZNONODE))
|
code == ZNONODE))
|
||||||
throw KeeperException(code, path);
|
throw KeeperException(code, path);
|
||||||
|
|
||||||
@ -418,6 +411,17 @@ void ZooKeeper::set(const std::string & path, const std::string & data, int32_t
|
|||||||
check(trySet(path, data, version, stat), path);
|
check(trySet(path, data, version, stat), path);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void ZooKeeper::createOrUpdate(const std::string & path, const std::string & data, int32_t mode)
|
||||||
|
{
|
||||||
|
int code = trySet(path, data, -1);
|
||||||
|
if (code == ZNONODE)
|
||||||
|
{
|
||||||
|
create(path, data, mode);
|
||||||
|
}
|
||||||
|
else if (code != ZOK)
|
||||||
|
throw zkutil::KeeperException(code, path);
|
||||||
|
}
|
||||||
|
|
||||||
int32_t ZooKeeper::trySet(const std::string & path, const std::string & data,
|
int32_t ZooKeeper::trySet(const std::string & path, const std::string & data,
|
||||||
int32_t version, Stat * stat_)
|
int32_t version, Stat * stat_)
|
||||||
{
|
{
|
||||||
|
Loading…
Reference in New Issue
Block a user